From bb43d23ad36ec3519a06d05c25ca99f726244ad0 Mon Sep 17 00:00:00 2001 From: Andy Coates <8012398+big-andy-coates@users.noreply.github.com> Date: Mon, 18 May 2020 14:07:43 +0100 Subject: [PATCH] feat: drop WITH(KEY) syntax (#5363) * feat: drop WITH(KEY) syntax fixes: https://github.com/confluentinc/ksql/issues/3537 implements: See [KLIP-25](https://github.com/confluentinc/ksql/blob/master/design-proposals/klip-25-removal-of-with-key-syntax.md) This change removes the `WITH(KEY)` syntax which previously allowed users to specify a value column that could act as an alias for the key column. This allowed a more user friendly name to be used for the key column, at the expense of requiring a copy of the key data in the value. With the new 'any key name' feature, the key columns themselves can be given appropriate names, removing the need for this aliasing functionality. See [KLIP-25](https://github.com/confluentinc/ksql/blob/master/design-proposals/klip-25-removal-of-with-key-syntax.md) for more details. Co-authored-by: Andy Coates --- docs/concepts/collections/streams.md | 3 +- docs/concepts/collections/tables.md | 3 +- docs/concepts/stream-processing.md | 22 +- docs/developer-guide/create-a-stream.md | 76 +- docs/developer-guide/create-a-table.md | 69 +- docs/developer-guide/joins/partition-data.md | 89 ++- .../ksqldb-reference/insert-values.md | 41 +- .../ksqldb-reference/operators.md | 27 +- .../query-with-arrays-and-maps.md | 7 +- docs/developer-guide/syntax-reference.md | 91 +-- .../generate-custom-test-data.md | 12 +- docs/operate-and-deploy/capacity-planning.md | 10 +- .../installation/server-config/avro-schema.md | 29 +- .../installation/upgrading.md | 65 +- docs/tutorials/basics-docker.md | 209 ++--- docs/tutorials/embedded-connect.md | 16 +- docs/tutorials/event-driven-microservice.md | 3 +- docs/tutorials/examples.md | 44 +- .../confluent/ksql/cli/console/Console.java | 15 +- .../ksql/cli/console/JLineReader.java | 2 +- .../java/io/confluent/ksql/cli/CliTest.java | 183 ++--- .../ksql/cli/console/ConsoleTest.java | 7 - .../ksql/properties/with/CreateConfigs.java | 8 - .../ksql/schema/ksql/ColumnMatchers.java | 26 - .../ddl/commands/CreateSourceFactory.java | 31 +- .../ksql/ddl/commands/DdlCommandExec.java | 11 - .../confluent/ksql/engine/EngineExecutor.java | 53 +- .../ksql/engine/InsertValuesExecutor.java | 34 - .../confluent/ksql/physical/PhysicalPlan.java | 13 +- .../ksql/physical/PhysicalPlanBuilder.java | 4 +- .../ksql/planner/LogicalPlanner.java | 83 -- .../ksql/planner/plan/AggregateNode.java | 10 - .../ksql/planner/plan/DataSourceNode.java | 16 +- .../ksql/planner/plan/FilterNode.java | 6 - .../ksql/planner/plan/FlatMapNode.java | 6 - .../confluent/ksql/planner/plan/JoinNode.java | 18 - .../ksql/planner/plan/KsqlBareOutputNode.java | 10 - .../plan/KsqlStructuredDataOutputNode.java | 10 - .../confluent/ksql/planner/plan/PlanNode.java | 3 - .../ksql/planner/plan/ProjectNode.java | 9 - .../ksql/planner/plan/RepartitionNode.java | 9 - .../ksql/structured/SchemaKGroupedStream.java | 9 - .../ksql/structured/SchemaKGroupedTable.java | 5 +- .../ksql/structured/SchemaKSourceFactory.java | 48 +- .../ksql/structured/SchemaKStream.java | 100 +-- .../ksql/structured/SchemaKTable.java | 20 - .../ksql/analyzer/AnalyzerFunctionalTest.java | 22 +- .../ddl/commands/CreateSourceFactoryTest.java | 19 - .../ksql/ddl/commands/DdlCommandExecTest.java | 70 +- .../ksql/engine/InsertValuesExecutorTest.java | 169 ++-- .../confluent/ksql/engine/KsqlEngineTest.java | 13 +- .../integration/EndToEndIntegrationTest.java | 131 +--- .../integration/IntegrationTestHarness.java | 57 +- .../ksql/integration/JoinIntTest.java | 12 +- .../ksql/integration/JsonFormatTest.java | 27 +- .../SelectValueMapperIntegrationTest.java | 4 +- .../StreamsSelectAndProjectIntTest.java | 83 +- .../ksql/integration/UdfIntTest.java | 52 +- .../ksql/integration/WindowingIntTest.java | 4 +- .../physical/PhysicalPlanBuilderTest.java | 92 +-- .../ksql/planner/LogicalPlannerTest.java | 9 +- .../ksql/planner/plan/AggregateNodeTest.java | 24 +- .../ksql/planner/plan/DataSourceNodeTest.java | 26 +- .../ksql/planner/plan/JoinNodeTest.java | 23 - .../KsqlStructuredDataOutputNodeTest.java | 5 - .../ksql/planner/plan/PlanNodeTest.java | 6 - .../planner/plan/RepartitionNodeTest.java | 7 +- .../inference/SchemaRegisterInjectorTest.java | 2 - .../KsqlAuthorizationValidatorImplTest.java | 2 - .../structured/SchemaKGroupedStreamTest.java | 4 - .../structured/SchemaKGroupedTableTest.java | 2 - .../structured/SchemaKSourceFactoryTest.java | 15 +- .../ksql/structured/SchemaKStreamTest.java | 255 +----- .../ksql/structured/SchemaKTableTest.java | 224 +----- .../ksql/topic/SourceTopicsExtractorTest.java | 2 - .../ksql/topic/TopicCreateInjectorTest.java | 7 +- .../confluent/ksql/util/ItemDataProvider.java | 29 +- .../ksql/util/OrderDataProvider.java | 37 +- .../ksql/util/PageViewDataProvider.java | 31 +- .../confluent/ksql/util/TestDataProvider.java | 28 +- .../confluent/ksql/util/UserDataProvider.java | 23 +- .../io/confluent/ksql/datagen/DataGen.java | 22 +- .../ksql/datagen/DataGenProducer.java | 4 +- .../confluent/ksql/embedded/EmbeddedKsql.java | 4 +- .../src/main/resources/SampleQueries.sql | 4 +- .../ddl/commands/CreateSourceCommand.java | 46 +- .../ddl/commands/CreateStreamCommand.java | 6 +- .../ddl/commands/CreateTableCommand.java | 6 +- .../ddl/commands/CreateSourceCommandTest.java | 67 +- .../ksql/test/model/KeyFieldNode.java | 52 -- .../confluent/ksql/test/model/SourceNode.java | 15 +- .../model/matchers/MetaStoreMatchers.java | 38 - .../test/tools/conditions/PostConditions.java | 1 - .../ksql/test/model/SourceNodeTest.java | 1 - .../6.0.0_1589386990638/plan.json | 162 ++++ .../6.0.0_1589386990638/spec.json | 117 +++ .../6.0.0_1589386990638/topology | 40 + .../6.0.0_1589386987411/plan.json | 161 ++++ .../6.0.0_1589386987411/spec.json | 133 ++++ .../6.0.0_1589386987411/topology | 25 + .../6.0.0_1589386987557/plan.json | 161 ++++ .../6.0.0_1589386987557/spec.json | 190 +++++ .../6.0.0_1589386987557/topology | 25 + .../6.0.0_1589386987645/plan.json | 161 ++++ .../6.0.0_1589386987645/spec.json | 178 +++++ .../6.0.0_1589386987645/topology | 25 + .../6.0.0_1589386987762/plan.json | 161 ++++ .../6.0.0_1589386987762/spec.json | 180 +++++ .../6.0.0_1589386987762/topology | 25 + .../6.0.0_1589386990851/plan.json | 162 ++++ .../6.0.0_1589386990851/spec.json | 117 +++ .../6.0.0_1589386990851/topology | 40 + .../6.0.0_1589386989345/plan.json | 162 ++++ .../6.0.0_1589386989345/spec.json | 122 +++ .../6.0.0_1589386989345/topology | 40 + .../6.0.0_1589386987914/plan.json | 162 ++++ .../6.0.0_1589386987914/spec.json | 143 ++++ .../6.0.0_1589386987914/topology | 40 + .../6.0.0_1589386988079/plan.json | 162 ++++ .../6.0.0_1589386988079/spec.json | 209 +++++ .../6.0.0_1589386988079/topology | 40 + .../6.0.0_1589386988168/plan.json | 162 ++++ .../6.0.0_1589386988168/spec.json | 193 +++++ .../6.0.0_1589386988168/topology | 40 + .../6.0.0_1589386988249/plan.json | 162 ++++ .../6.0.0_1589386988249/spec.json | 195 +++++ .../6.0.0_1589386988249/topology | 40 + .../6.0.0_1589386988413/plan.json | 162 ++++ .../6.0.0_1589386988413/spec.json | 159 ++++ .../6.0.0_1589386988413/topology | 40 + .../6.0.0_1589386988647/plan.json | 162 ++++ .../6.0.0_1589386988647/spec.json | 236 ++++++ .../6.0.0_1589386988647/topology | 40 + .../6.0.0_1589386988760/plan.json | 162 ++++ .../6.0.0_1589386988760/spec.json | 220 ++++++ .../6.0.0_1589386988760/topology | 40 + .../6.0.0_1589386988869/plan.json | 162 ++++ .../6.0.0_1589386988869/spec.json | 222 ++++++ .../6.0.0_1589386988869/topology | 40 + .../6.0.0_1589386988518/plan.json | 162 ++++ .../6.0.0_1589386988518/spec.json | 159 ++++ .../6.0.0_1589386988518/topology | 40 + .../6.0.0_1589386990102/plan.json | 162 ++++ .../6.0.0_1589386990102/spec.json | 122 +++ .../6.0.0_1589386990102/topology | 40 + .../6.0.0_1589386989597/plan.json | 162 ++++ .../6.0.0_1589386989597/spec.json | 122 +++ .../6.0.0_1589386989597/topology | 40 + .../6.0.0_1589386989894/plan.json | 162 ++++ .../6.0.0_1589386989894/spec.json | 113 +++ .../6.0.0_1589386989894/topology | 40 + .../6.0.0_1589386991753/plan.json | 162 ++++ .../6.0.0_1589386991753/spec.json | 113 +++ .../6.0.0_1589386991753/topology | 40 + .../6.0.0_1589386991385/plan.json | 169 ++++ .../6.0.0_1589386991385/spec.json | 113 +++ .../6.0.0_1589386991385/topology | 49 ++ .../6.0.0_1589386988327/plan.json | 162 ++++ .../6.0.0_1589386988327/spec.json | 118 +++ .../6.0.0_1589386988327/topology | 40 + .../6.0.0_1589386993127/plan.json | 168 ++++ .../6.0.0_1589386993127/spec.json | 103 +++ .../6.0.0_1589386993127/topology | 34 + .../6.0.0_1589386993055/plan.json | 173 +++++ .../6.0.0_1589386993055/spec.json | 110 +++ .../6.0.0_1589386993055/topology | 37 + .../6.0.0_1589386993189/plan.json | 161 ++++ .../6.0.0_1589386993189/spec.json | 147 ++++ .../6.0.0_1589386993189/topology | 25 + .../6.0.0_1589386993254/plan.json | 161 ++++ .../6.0.0_1589386993254/spec.json | 135 ++++ .../6.0.0_1589386993254/topology | 25 + .../6.0.0_1589386993323/plan.json | 161 ++++ .../6.0.0_1589386993323/spec.json | 137 ++++ .../6.0.0_1589386993323/topology | 25 + .../6.0.0_1589386994011/plan.json | 167 ++++ .../6.0.0_1589386994011/spec.json | 219 ++++++ .../6.0.0_1589386994011/topology | 43 ++ .../6.0.0_1589386993744/plan.json | 166 ++++ .../6.0.0_1589386993744/spec.json | 295 +++++++ .../6.0.0_1589386993744/topology | 28 + .../6.0.0_1589386993815/plan.json | 166 ++++ .../6.0.0_1589386993815/spec.json | 146 ++++ .../6.0.0_1589386993815/topology | 28 + .../6.0.0_1589386993890/plan.json | 166 ++++ .../6.0.0_1589386993890/spec.json | 123 +++ .../6.0.0_1589386993890/topology | 28 + .../6.0.0_1589386993956/plan.json | 166 ++++ .../6.0.0_1589386993956/spec.json | 155 ++++ .../6.0.0_1589386993956/topology | 28 + .../6.0.0_1589386994151/plan.json | 197 +++++ .../6.0.0_1589386994151/spec.json | 108 +++ .../6.0.0_1589386994151/topology | 39 + .../6.0.0_1589386994217/plan.json | 197 +++++ .../6.0.0_1589386994217/spec.json | 108 +++ .../6.0.0_1589386994217/topology | 39 + .../6.0.0_1589386994303/plan.json | 197 +++++ .../6.0.0_1589386994303/spec.json | 108 +++ .../6.0.0_1589386994303/topology | 39 + .../6.0.0_1589386994372/plan.json | 197 +++++ .../6.0.0_1589386994372/spec.json | 108 +++ .../6.0.0_1589386994372/topology | 39 + .../6.0.0_1589387013555/plan.json | 131 ++++ .../6.0.0_1589387013555/spec.json | 67 ++ .../6.0.0_1589387013555/topology | 16 + .../6.0.0_1589387013813/plan.json | 204 +++++ .../6.0.0_1589387013813/spec.json | 115 +++ .../6.0.0_1589387013813/topology | 42 + .../6.0.0_1589387013690/plan.json | 197 +++++ .../6.0.0_1589387013690/spec.json | 110 +++ .../6.0.0_1589387013690/topology | 39 + .../6.0.0_1589387013749/plan.json | 218 ++++++ .../6.0.0_1589387013749/spec.json | 129 ++++ .../6.0.0_1589387013749/topology | 66 ++ .../6.0.0_1589387013877/plan.json | 218 ++++++ .../6.0.0_1589387013877/spec.json | 134 ++++ .../6.0.0_1589387013877/topology | 66 ++ .../6.0.0_1589387013948/plan.json | 211 +++++ .../6.0.0_1589387013948/spec.json | 134 ++++ .../6.0.0_1589387013948/topology | 63 ++ .../6.0.0_1589387013598/plan.json | 131 ++++ .../6.0.0_1589387013598/spec.json | 67 ++ .../6.0.0_1589387013598/topology | 16 + .../6.0.0_1589387014881/plan.json | 194 +++++ .../6.0.0_1589387014881/spec.json | 106 +++ .../6.0.0_1589387014881/topology | 42 + .../6.0.0_1589387015878/plan.json | 166 ++++ .../6.0.0_1589387015878/spec.json | 170 ++++ .../6.0.0_1589387015878/topology | 43 ++ .../comparison-expression.json | 12 +- .../query-validation-tests/group-by.json | 80 +- .../query-validation-tests/having.json | 20 +- .../query-validation-tests/histogram.json | 12 +- .../hopping-windows.json | 40 +- .../query-validation-tests/identifiers.json | 32 +- .../query-validation-tests/insert-into.json | 2 +- .../query-validation-tests/key-field.json | 727 ------------------ .../query-validation-tests/key-schemas.json | 1 - .../query-validation-tests/partition-by.json | 36 +- .../quoted-identifiers.json | 14 +- .../session-windows.json | 2 +- .../insert-values.json | 78 +- .../correct/multi_join/statements.sql | 15 +- .../incorrect-test6/statements.sql | 2 +- .../incorrect/empty_input/statements.sql | 2 +- .../incorrect/empty_output/statements.sql | 2 +- .../incorrect_input_format/statements.sql | 2 +- .../missing_field_in_output/statements.sql | 2 +- .../test-runner/test4/statements.sql | 3 +- .../test/resources/testing_tool_tests.json | 8 +- .../ksql/metastore/model/DataSource.java | 5 - .../ksql/metastore/model/KeyField.java | 140 ---- .../ksql/metastore/model/KsqlStream.java | 2 - .../ksql/metastore/model/KsqlTable.java | 2 - .../metastore/model/StructuredDataSource.java | 9 - .../ksql/metastore/model/KeyFieldTest.java | 166 ---- .../metastore/model/MetaStoreMatchers.java | 71 -- .../metastore/model/MetaStoreModelTest.java | 2 - .../model/StructuredDataSourceTest.java | 50 +- .../confluent/ksql/util/MetaStoreFixture.java | 30 +- .../with/CreateSourceProperties.java | 5 - .../confluent/ksql/parser/KsqlParserTest.java | 22 +- .../ksql/parser/SqlFormatterTest.java | 16 +- .../with/CreateSourcePropertiesTest.java | 14 - .../rest/entity/SourceDescriptionFactory.java | 1 - .../api/integration/ApiIntegrationTest.java | 18 +- .../entity/SourceDescriptionFactoryTest.java | 2 - .../healthcheck/HealthCheckAgentTest.java | 3 +- .../AuthorizationFunctionalTest.java | 2 - .../KsqlResourceFunctionalTest.java | 10 +- .../LagReportingAgentFunctionalTest.java | 6 +- .../integration/PullQueryFunctionalTest.java | 1 - .../PullQueryRoutingFunctionalTest.java | 1 - .../ksql/rest/integration/RestApiTest.java | 8 +- ...QueriesMultiNodeWithTlsFunctionalTest.java | 8 +- .../ksql/rest/server/KsqlPlanSchemaTest.java | 1 + .../StandaloneExecutorFunctionalTest.java | 8 +- .../ksql/rest/server/TemporaryEngine.java | 3 - .../DescribeConnectorExecutorTest.java | 2 - .../server/resources/KsqlResourceTest.java | 17 - .../resources/ksql-plan-schema/schema.json | 6 - .../ksql/rest/entity/SourceDescription.java | 9 - .../rest/entity/SourceDescriptionTest.java | 41 +- .../util/EmbeddedSingleNodeKafkaCluster.java | 47 +- 284 files changed, 17000 insertions(+), 4034 deletions(-) delete mode 100644 ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/KeyFieldNode.java create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1589387015878/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1589387015878/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1589387015878/topology delete mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/key-field.json delete mode 100644 ksqldb-metastore/src/main/java/io/confluent/ksql/metastore/model/KeyField.java delete mode 100644 ksqldb-metastore/src/test/java/io/confluent/ksql/metastore/model/KeyFieldTest.java diff --git a/docs/concepts/collections/streams.md b/docs/concepts/collections/streams.md index b08013af44e4..2c4ba5bd487d 100644 --- a/docs/concepts/collections/streams.md +++ b/docs/concepts/collections/streams.md @@ -26,10 +26,9 @@ in the `publications` stream are distributed over 3 partitions, are keyed on the `author` column, and are serialized in the Avro format. ```sql -CREATE STREAM publications (author VARCHAR, title VARCHAR) +CREATE STREAM publications (author VARCHAR KEY, title VARCHAR) WITH (kafka_topic = 'publication_events', partitions = 3, - key = 'author', value_format = 'avro'); ``` diff --git a/docs/concepts/collections/tables.md b/docs/concepts/collections/tables.md index b3ff0c6dc4e6..f877863198a7 100644 --- a/docs/concepts/collections/tables.md +++ b/docs/concepts/collections/tables.md @@ -42,9 +42,8 @@ partitions, are keyed on the `title` column, and are serialized in the Avro format. ```sql -CREATE TABLE movies (title VARCHAR, release_year INT) +CREATE TABLE movies (title VARCHAR PRIMARY KEY, release_year INT) WITH (kafka_topic = 'movies', - key = 'title' partitions = 5, value_format = 'avro'); ``` diff --git a/docs/concepts/stream-processing.md b/docs/concepts/stream-processing.md index 1c4613bd9e82..630ed199f319 100644 --- a/docs/concepts/stream-processing.md +++ b/docs/concepts/stream-processing.md @@ -63,14 +63,14 @@ Deriving a new table from an existing stream Given the following table and stream: ```sql -CREATE TABLE products (product_name VARCHAR, cost DOUBLE) - WITH (kafka_topic='products', partitions=1, value_format='json', key='product_name'); +CREATE TABLE products (product_name VARCHAR PRIMARY KEY, cost DOUBLE) + WITH (kafka_topic='products', partitions=1, value_format='json'); -CREATE STREAM orders (product_name VARCHAR) - WITH (kafka_topic='orders', partitions=1, value_format='json', key='product_name'); +CREATE STREAM orders (product_name VARCHAR KEY) + WITH (kafka_topic='orders', partitions=1, value_format='json'); ``` -You can create an table that aggregates rows from the `orders` stream, while +You can create a table that aggregates rows from the `orders` stream, while also joining the stream on the `products` table to enrich the `orders` data: ```sql @@ -109,11 +109,11 @@ Deriving a new stream from multiple streams Given the following two streams: ```sql -CREATE STREAM impressions (user VARCHAR, impression_id BIGINT, url VARCHAR) - WITH (kafka_topic='impressions', partitions=1, value_format='json', key='user'); +CREATE STREAM impressions (user VARCHAR KEY, impression_id BIGINT, url VARCHAR) + WITH (kafka_topic='impressions', partitions=1, value_format='json'); -CREATE STREAM clicks (user VARCHAR) - WITH (kafka_topic='clicks', partitions=1, value_format='json', key='user'); +CREATE STREAM clicks (user VARCHAR KEY, url VARCHAR) + WITH (kafka_topic='clicks', partitions=1, value_format='json'); ``` You can create a derived stream that joins the `impressions` and `clicks` @@ -122,7 +122,9 @@ within one minute of the initial ad impression: ```sql CREATE STREAM clicked_impressions AS - SELECT * FROM impressions i JOIN clicks c WITHIN 1 minute ON i.user = c.user EMIT CHANGES; + SELECT * FROM impressions i JOIN clicks c WITHIN 1 minute ON i.user = c.user + WHERE i.url = c.url + EMIT CHANGES; ``` Any time an `impressions` row is received, followed within one minute by a diff --git a/docs/developer-guide/create-a-stream.md b/docs/developer-guide/create-a-stream.md index 9b4cd65e4efb..af8245a0ae5b 100644 --- a/docs/developer-guide/create-a-stream.md +++ b/docs/developer-guide/create-a-stream.md @@ -98,30 +98,34 @@ Name : PAGEVIEWS For runtime statistics and query details run: DESCRIBE EXTENDED ; ``` +You may notice that ksqlDB has added a key column named `ROWKEY`. +This is the default key column that ksqlDB adds if you don't provide one. +If your data doesn't contain a {{ site.ak }} serialized +`STRING` in the {{ site.ak }} message key, don't use `ROWKEY` in your SQL statements, +because this may cause unexpected results. + ### Create a Stream with a Specified Key -The previous SQL statement makes no assumptions about the Kafka message -key in the underlying Kafka topic. If the value of the message key in -the topic is the same as one of the columns defined in the stream, you -can specify the key in the WITH clause of the CREATE STREAM statement. -If you use this column name later to perform a join or a repartition, ksqlDB -knows that no repartition is needed. In effect, the named column becomes an -alias for ROWKEY. +The previous SQL statement doesn't define a column to represent the data in the +{{ site.ak }} message key in the underlying {{ site.ak }} topic, so the system adds a +`ROWKEY` column with type `STRING`. + +If the {{ site.ak }} message key is serialized in a key format that ksqlDB supports (currently `KAFKA`), +you can specify the key in the column list of the CREATE STREAM statement. -For example, if the Kafka message key has the same value as the `pageid` -column, you can write the CREATE STREAM statement like this: +For example, the {{ site.ak }} message key of the `pageviews` topic is a `BIGINT` containing the `viewtime`, +so you can write the CREATE STREAM statement like this: ```sql CREATE STREAM pageviews_withkey - (viewtime BIGINT, + (viewtime BIGINT KEY, userid VARCHAR, pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews', - VALUE_FORMAT='DELIMITED', - KEY='pageid'); + VALUE_FORMAT='DELIMITED'); ``` -Confirm that the KEY field in the new stream is `pageid` by using the +Confirm that the KEY column in the new stream is `pageid` by using the DESCRIBE EXTENDED statement: ```sql @@ -133,11 +137,17 @@ Your output should resemble: ``` Name : PAGEVIEWS_WITHKEY Type : STREAM -Key field : PAGEID -Key format : STRING Timestamp field : Not set - using +Key format : KAFKA Value format : DELIMITED Kafka topic : pageviews (partitions: 1, replication: 1) + + Field | Type +-------------------------------------- + VIEWTIME | BIGINT (Key) + USERID | VARCHAR(STRING) + PAGEID | VARCHAR(STRING) +-------------------------------------- [...] ``` @@ -155,12 +165,11 @@ like this: ```sql CREATE STREAM pageviews_timestamped - (viewtime BIGINT, - userid VARCHAR, + (viewtime BIGINT KEY, + userid VARCHAR pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews', VALUE_FORMAT='DELIMITED', - KEY='pageid', TIMESTAMP='viewtime') ``` @@ -176,9 +185,8 @@ Your output should resemble: ``` Name : PAGEVIEWS_TIMESTAMPED Type : STREAM -Key field : PAGEID -Key format : STRING Timestamp field : VIEWTIME +Key format : KAFKA Value format : DELIMITED Kafka topic : pageviews (partitions: 1, replication: 1) [...] @@ -197,7 +205,7 @@ the following CREATE STREAM statement into the CLI: ```sql CREATE STREAM pageviews - (viewtime BIGINT, + (viewtime BIGINT KEY, userid VARCHAR, pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews', @@ -242,12 +250,6 @@ them explicitly. To stream the result of a SELECT query into an *existing* stream and its underlying topic, use the INSERT INTO statement. -!!! note - The CREATE STREAM AS SELECT statement doesn't support the KEY property. - To specify a KEY field, use the PARTITION BY clause. For more - information, see - [Partition Data to Enable Joins](joins/partition-data.md). - The following SQL statement creates a `pageviews_intro` stream that contains results from a persistent query that matches "introductory" pages that have a `pageid` value that's less than `Page_20`: @@ -280,12 +282,12 @@ Your output should resemble: ``` Key format: KAFKA_BIGINT or KAFKA_DOUBLE Value format: KAFKA_STRING -rowtime: 10/30/18 10:15:51 PM GMT, key: 294851, value: 1540937751186,User_8,Page_12 -rowtime: 10/30/18 10:15:55 PM GMT, key: 295051, value: 1540937755255,User_1,Page_15 -rowtime: 10/30/18 10:15:57 PM GMT, key: 295111, value: 1540937757265,User_8,Page_10 -rowtime: 10/30/18 10:15:59 PM GMT, key: 295221, value: 1540937759330,User_4,Page_15 -rowtime: 10/30/18 10:15:59 PM GMT, key: 295231, value: 1540937759699,User_1,Page_12 -rowtime: 10/30/18 10:15:59 PM GMT, key: 295241, value: 1540937759990,User_6,Page_15 +rowtime: 10/30/18 10:15:51 PM GMT, key: 1540937751186, value: 1540937751186,User_8,Page_12 +rowtime: 10/30/18 10:15:55 PM GMT, key: 1540937755255, value: 1540937755255,User_1,Page_15 +rowtime: 10/30/18 10:15:57 PM GMT, key: 1540937757265, value: 1540937757265,User_8,Page_10 +rowtime: 10/30/18 10:15:59 PM GMT, key: 1540937759330, value: 1540937759330,User_4,Page_15 +rowtime: 10/30/18 10:15:59 PM GMT, key: 1540937759699, value: 1540937759699,User_1,Page_12 +rowtime: 10/30/18 10:15:59 PM GMT, key: 1540937759990, value: 1540937759990,User_6,Page_15 ^CTopic printing ceased ``` @@ -294,13 +296,6 @@ Press Ctrl+C to stop printing the stream. !!! note The query continues to run after you stop printing the stream. -!!! note - KsqlDB has determined that the key format is either `KAFKA_BIGINT` or `KAFKA_DOUBLE`. - KsqlDB has not narrowed it further because it is not possible to rule out - either format just by inspecting the key's serialized bytes. In this case we know the key is - a `BIGINT`. For other cases you may know the key type or you may need to speak to the author - of the data. - Use the SHOW QUERIES statement to view the query that ksqlDB created for the `pageviews_intro` stream: @@ -364,4 +359,3 @@ Next Steps ---------- - [Join Event Streams with ksqlDB](joins/join-streams-and-tables.md) - diff --git a/docs/developer-guide/create-a-table.md b/docs/developer-guide/create-a-table.md index 08ff9cc25ee1..f53417fe4bf5 100644 --- a/docs/developer-guide/create-a-table.md +++ b/docs/developer-guide/create-a-table.md @@ -52,13 +52,12 @@ In the ksqlDB CLI, paste the following CREATE TABLE statement: ```sql CREATE TABLE users - (registertime BIGINT, - userid VARCHAR, + (userid VARCHAR PRIMARY KEY, + registertime BIGINT, gender VARCHAR, regionid VARCHAR) WITH (KAFKA_TOPIC = 'users', - VALUE_FORMAT='JSON', - KEY = 'userid'); + VALUE_FORMAT='JSON'); ``` Your output should resemble: @@ -97,9 +96,8 @@ Your output should resemble: Name : USERS Field | Type ------------------------------------------ - ROWKEY | VARCHAR(STRING) (key) + USERID | VARCHAR(STRING) (key) REGISTERTIME | BIGINT - USERID | VARCHAR(STRING) GENDER | VARCHAR(STRING) REGIONID | VARCHAR(STRING) ------------------------------------------ @@ -116,13 +114,13 @@ SELECT * FROM users EMIT CHANGES; Assuming the table has content, your output should resemble: ``` -+--------+---------------+--------+--------+----------+ -| ROWKEY | REGISTERTIME | USERID | GENDER | REGIONID | -+--------+---------------+--------+--------+----------+ -| User_2 | 1498028899054 | User_2 | MALE | Region_1 | -| User_6 | 1505677113995 | User_6 | FEMALE | Region_7 | -| User_5 | 1491338621627 | User_5 | OTHER | Region_2 | -| User_9 | 1492621173463 | User_9 | FEMALE | Region_3 | ++--------+---------------+--------+----------+ +| USERID | REGISTERTIME | GENDER | REGIONID | ++--------+---------------+--------+----------+ +| User_2 | 1498028899054 | MALE | Region_1 | +| User_6 | 1505677113995 | FEMALE | Region_7 | +| User_5 | 1491338621627 | OTHER | Region_2 | +| User_9 | 1492621173463 | FEMALE | Region_3 | ^CQuery terminated ``` @@ -144,15 +142,14 @@ the following CREATE TABLE statement into the CLI: ```sql CREATE TABLE users - (registertime BIGINT, - userid VARCHAR, + (userid VARCHAR PRIMARY KEY, + registertime BIGINT, gender VARCHAR, regionid VARCHAR) WITH (KAFKA_TOPIC = 'users', VALUE_FORMAT='JSON', PARTITIONS=4, - REPLICAS=3 - KEY = 'userid'); + REPLICAS=3); ``` This will create the users topics for you with the supplied partition and replica count. @@ -216,9 +213,9 @@ Your output should resemble: ``` Key format: KAFKA_STRING Value format: JSON -rowTime: 12/21/18 23:58:42 PM PSD, key: User_5, value: {"USERID":"User_5","GENDER":"FEMALE","REGIONID":"Region_4"} -rowTime: 12/21/18 23:58:42 PM PSD, key: User_2, value: {"USERID":"User_2","GENDER":"FEMALE","REGIONID":"Region_7"} -rowTime: 12/21/18 23:58:42 PM PSD, key: User_9, value: {"USERID":"User_9","GENDER":"FEMALE","REGIONID":"Region_4"} +rowTime: 12/21/18 23:58:42 PM PSD, key: User_5, value: {"GENDER":"FEMALE","REGIONID":"Region_4"} +rowTime: 12/21/18 23:58:42 PM PSD, key: User_2, value: {"GENDER":"FEMALE","REGIONID":"Region_7"} +rowTime: 12/21/18 23:58:42 PM PSD, key: User_9, value: {"GENDER":"FEMALE","REGIONID":"Region_4"} ^CTopic printing ceased ``` @@ -282,17 +279,17 @@ SELECT ROWTIME, * FROM pageviews_table EMIT CHANGES; Your output should resemble: ``` -+---------------+---------------+---------------+------------------+--------+---------+------+ -| ROWTIME | WINDOWSTART | WINDOWEND | ROWKEY | USERID | PAGEID | TOTAL| -+---------------+---------------+---------------+------------------+--------+---------+------+ -| 1557183919786 | 1557183900000 | 1557183960000 | User_5|+|Page_12 | User_5 | Page_12 | 1 | -| 1557183929488 | 1557183900000 | 1557183960000 | User_9|+|Page_39 | User_9 | Page_39 | 1 | -| 1557183930211 | 1557183900000 | 1557183960000 | User_1|+|Page_79 | User_1 | Page_79 | 1 | -| 1557183930687 | 1557183900000 | 1557183960000 | User_9|+|Page_34 | User_9 | Page_34 | 1 | -| 1557183929786 | 1557183900000 | 1557183960000 | User_5|+|Page_12 | User_5 | Page_12 | 2 | -| 1557183931095 | 1557183900000 | 1557183960000 | User_3|+|Page_43 | User_3 | Page_43 | 1 | -| 1557183930184 | 1557183900000 | 1557183960000 | User_1|+|Page_29 | User_1 | Page_29 | 1 | -| 1557183930727 | 1557183900000 | 1557183960000 | User_6|+|Page_93 | User_6 | Page_93 | 3 | ++---------------+---------------+---------------+------------------+------+ +| ROWTIME | WINDOWSTART | WINDOWEND | KSQL_COL_0 | TOTAL| ++---------------+---------------+---------------+------------------+------+ +| 1557183919786 | 1557183900000 | 1557183960000 | User_5|+|Page_12 | 1 | +| 1557183929488 | 1557183900000 | 1557183960000 | User_9|+|Page_39 | 1 | +| 1557183930211 | 1557183900000 | 1557183960000 | User_1|+|Page_79 | 1 | +| 1557183930687 | 1557183900000 | 1557183960000 | User_9|+|Page_34 | 1 | +| 1557183929786 | 1557183900000 | 1557183960000 | User_5|+|Page_12 | 2 | +| 1557183931095 | 1557183900000 | 1557183960000 | User_3|+|Page_43 | 1 | +| 1557183930184 | 1557183900000 | 1557183960000 | User_1|+|Page_29 | 1 | +| 1557183930727 | 1557183900000 | 1557183960000 | User_6|+|Page_93 | 3 | ^CQuery terminated ``` @@ -304,16 +301,16 @@ Look up the value for a specific key within the table by using a SELECT statement. ```sql -SELECT * FROM pageviews_table WHERE ROWKEY='User_9|+|Page_39'; +SELECT * FROM pageviews_table WHERE KSQL_COL_0='User_9|+|Page_39'; ``` Your output should resemble: ``` -+------------------+---------------+---------------+--------+---------+-------+ -| ROWKEY | WINDOWSTART | WINDOWEND | USERID | PAGEID | TOTAL | -+------------------+---------------+---------------+--------+---------+-------+ -| User_9|+|Page_39 | 1557183900000 | 1557183960000 | User_9 | Page_39 | 1 | ++------------------+---------------+---------------+--------+ +| KSQL_COL_0 | WINDOWSTART | WINDOWEND | TOTAL | ++------------------+---------------+---------------+--------+ +| User_9|+|Page_39 | 1557183900000 | 1557183960000 | 1 | Query terminated ``` diff --git a/docs/developer-guide/joins/partition-data.md b/docs/developer-guide/joins/partition-data.md index 2e9add7ba0cf..ceffa906c5be 100644 --- a/docs/developer-guide/joins/partition-data.md +++ b/docs/developer-guide/joins/partition-data.md @@ -25,61 +25,71 @@ of tables, so you can only use a table's primary key as a join column. Streams, on the other hand, may not have a defined key or may have a key that differs from the join column. In these cases, ksqlDB internally repartitions -the stream, which implicitly defines a key for it. +the stream, which implicitly defines the correct key for it. + +!!! important + {{ site.ak }} guarantees the relative order of any two messages from + one source partition only if they are both in the same partition + after the repartition. Otherwise, {{ site.ak }} is likely to interleave messages. + The use case will determine if these ordering guarantees are acceptable. ksqlDB requires keys to use the `KAFKA` format. For more information, see [Serialization Formats](../serialization.md#serialization-formats). If internally repartitioning, ksqlDB uses the correct format. Because you can only use the primary key of a table as a joining column, it's -important to understand how keys are defined. For both streams and tables, the -column that represents the key has the name `ROWKEY`. +important to understand how keys are defined. -When you create a table by using a CREATE TABLE statement, the key of the -table is the same as that of the records in the underlying Kafka topic. -You must set the type of the `ROWKEY` column in the -CREATE TABLE statement to match the key data in the underlying {{ site.ak }} topic. +When you create a table by using a CREATE TABLE statement you define the key in the schema and +it must be the same as that of the records in the underlying {{ site.ak }} topic. As the +KAFKA format does not support named fields that key has no implicit name, so the key can be +given any name in the schema definition. When you create a table by using a CREATE TABLE AS SELECT statement, the key of the resulting table is determined as follows: -- If the FROM clause contains a stream, the statement must have a GROUP BY clause, - and the grouping columns determine the key of the resulting table. - - When grouping by a single column or expression, the type of `ROWKEY` in the - resulting stream matches the type of the column or expression. - - When grouping by multiple columns or expressions, the type of `ROWKEY` in the - resulting stream is an [SQL `STRING`](../../concepts/schemas). -- If the FROM clause contains only tables and no GROUP BY clause, the key is - copied over from the key of the table(s) in the FROM clause. -- If the FROM clause contains only tables and has a GROUP BY clause, the - grouping columns determine the key of the resulting table. - - When grouping by a single column or expression, the type of `ROWKEY` in the - resulting stream matches the type of the column or expression. - - When grouping by multiple columns or expressions, the type of `ROWKEY` in the - resulting stream is an [SQL `STRING`](../../concepts/schemas). +- If the statement contains neither a JOIN or GROUP BY clause, the key type of the resulting + table matches the key type of the source table, and the name matches the source unless the + projection defines an alias for the column. +- If the statement contains a JOIN and no GROUP BY clause, the key type of the resulting table + will match the type of the join columns and then key name will: + - FULL OUTER joins and joins on expressions other than column references will have a system + generated name in the form `KSQL_COL_n`, where `n` is a positive integer, unless the projection + defines an alias for the column. + - For other joins that contain at least one column reference in their join criteria, the name + will match the left most column reference in the join criteria. +- If the statement contains a GROUP BY clause, the grouping columns determine the key + of the resulting table. + - When grouping by a single column or STRUCT field: the name of the key column in the + resulting table matches the name of the column or field, unless the projection includes + an alias for the column or field, and the type of the key column matches the column or field. + - When grouping by a single expression that is not a column or STRUCT field: the resulting table + will have a system generated key column name in the form `KSQL_COL_n`, where `n` is a positive + integer, unless the projection contains an alias for the expression, and the type of the + column will match the result of the expression. + - When grouping by multiple expressions: the resulting table will have a system generated key + name in the form `KSQL_COL_n`, where `n` is a positive integer, and the type of the column will + be a [SQL `STRING`](../../concepts/schemas), containing the grouping expressions concatenated + together. The following example shows a `users` table joined with a `clicks` stream on the `userId` column. The `users` table has the correct primary key `userId` that coincides with the joining column. But the `clicks` stream -doesn't have a defined key, and ksqlDB must repartition it on the joining -column (`userId`) and assign the key before performing the join. +doesn't have a defined key, so ksqlDB must repartition it on the joining +column (`userId`) to assign the key before performing the join. ```sql -- clicks stream, with an unknown key. - -- the schema of stream clicks is: ROWKEY STRING | USERID BIGINT | URL STRING + -- the schema of stream clicks is: ROWKEY STRING KEY | USERID BIGINT | URL STRING CREATE STREAM clicks (userId BIGINT, url STRING) WITH(kafka_topic='clickstream', value_format='json'); -- the primary key of table users is a BIGINT. - -- The userId column in the value matches the key, so can be used as an alias for ROWKEY in queries to make them more readable. - -- the schema of table users is: ROWKEY BIGINT | USERID BIGINT | FULLNAME STRING - CREATE TABLE users (ROWKEY BIGINT PRIMARY KEY, userId BIGINT, fullName STRING) WITH(kafka_topic='users', value_format='json', key='userId'); + -- the schema of table users is: USERID BIGINT KEY | FULLNAME STRING + CREATE TABLE users (userId BIGINT PRIMARY KEY, fullName STRING) WITH(kafka_topic='users', value_format='json'); - -- join of users table with clicks stream, joining on the table's primary key alias and the stream's userId column: + -- join of users table with clicks stream, joining on the table's primary key and the stream's userId column: -- join will automatically repartition clicks stream: SELECT clicks.url, users.fullName FROM clicks JOIN users ON clicks.userId = users.userId; - - -- The following is equivalent and does not rely on their being a copy of the tables key within the value schema: - SELECT clicks.url, users.fullName FROM clicks JOIN users ON clicks.userId = users.ROWKEY; ``` Co-partitioning Requirements @@ -90,7 +100,7 @@ and tables are *co-partitioned*, which means that input records on both sides of the join have the same configuration settings for partitions. - The input records for the join must have the - [same keying schema](#records-have-the-same-keying-schema). + [same key schema](#records-have-the-same-key-schema). - The input records must have the [same number of partitions](#records-have-the-same-number-of-partitions) on both sides. @@ -101,7 +111,7 @@ When your inputs are co-partitioned, records with the same key, from both sides of the join, are delivered to the same stream task during processing. -### Records Have the Same Keying Schema +### Records Have the Same Key Schema For a join to work, the keys from both sides must have the same SQL type. @@ -116,24 +126,23 @@ the `INT` side to a `LONG`: ```sql -- stream with INT userId - CREATE STREAM clicks (userId INT, url STRING) WITH(kafka_topic='clickstream', value_format='json'); + CREATE STREAM clicks (userId INT KEY, url STRING) WITH(kafka_topic='clickstream', value_format='json'); - -- table with BIGINT userId stored in they key: - CREATE TABLE users (ROWKEY BIGINT PRIMARY KEY, fullName STRING) WITH(kafka_topic='users', value_format='json'); + -- table with BIGINT id stored in the key: + CREATE TABLE users (id BIGINT PRIMARY KEY, fullName STRING) WITH(kafka_topic='users', value_format='json'); -- Join utilising a CAST to convert the left sides join column to match the rights type. - SELECT clicks.url, users.fullName FROM clicks JOIN users ON CAST(clicks.userId AS BIGINT) = users.ROWKEY; + SELECT clicks.url, users.fullName FROM clicks JOIN users ON CAST(clicks.userId AS BIGINT) = users.id; ``` - Tables created on top of existing Kafka topics, for example those created with a `CREATE TABLE` statement, are keyed on the data held in the key of the records -in the Kafka topic. ksqlDB presents this data in the `ROWKEY` column and expects +in the Kafka topic. ksqlDB presents this data in the `PRIMARY KEY` column and expects the data to be in the `KAFKA` format. Tables created inside ksqlDB from other sources, for example those created with a `CREATE TABLE AS SELECT` statement, will copy the key from their source(s) -unless there is an explicit `GROUP BY` clause, which can change what the table +unless there is an explicit `GROUP BY` or `JOIN` clause, which can change what the table is keyed on. !!! note diff --git a/docs/developer-guide/ksqldb-reference/insert-values.md b/docs/developer-guide/ksqldb-reference/insert-values.md index 04c9ba2107d0..ba2bf7415050 100644 --- a/docs/developer-guide/ksqldb-reference/insert-values.md +++ b/docs/developer-guide/ksqldb-reference/insert-values.md @@ -21,18 +21,14 @@ Description ----------- Produce a row into an existing stream or table and its underlying topic -based on explicitly specified values. The first `column_name` of every -schema is `ROWKEY`, which defines the corresponding Kafka key. If the -source specifies a `key` and that column is present in the column names -for this INSERT statement then that value and the `ROWKEY` value are -expected to match, otherwise the value from `ROWKEY` will be copied into -the value of the key column (or conversely from the key column into the -`ROWKEY` column). - -Any column not explicitly given a value is set to `null`. If no columns -are specified, a value for every column is expected in the same order as -the schema with `ROWKEY` as the first column. If columns are specified, -the order does not matter. +based on explicitly specified values. + +If column names are specified, then the order of the values must match the +order of the names. Any column not explicitly given a value is set to `null`. +Pseudo columns, for example `ROWTIME`, may be provided. + +If no columns are specified, a value for every column is expected in the same +order as the schema, with key columns first. !!! note `ROWTIME` may be specified as an explicit column but isn't required @@ -43,21 +39,24 @@ Example ------- The following statements are valid for a source with a schema like -`` with `KEY=KEY_COL`. +`KEY_COL VARCHAR KEY, COL_A VARCHAR`. ```sql --- inserts (1234, "key", "key", "A") -INSERT INTO foo (ROWTIME, ROWKEY, KEY_COL, COL_A) VALUES (1510923225000, 'key', 'key', 'A'); +-- inserts (ROWTIME:=1510923225000, KEY_COL:="key", COL_A:="A") +INSERT INTO foo (ROWTIME, KEY_COL, COL_A) VALUES (1510923225000, 'key', 'A'); + +-- also inserts (ROWTIME:=1510923225000, KEY_COL:="key", COL_A:="A") +INSERT INTO foo (COL_A, ROWTIME, KEY_COL) VALUES ('A', 1510923225000, 'key'); --- inserts (current_time(), "key", "key", "A") -INSERT INTO foo VALUES ('key', 'key', 'A'); +-- inserts (ROWTIME:=current_time(), KEY_COL:="key", COL_A:="A") +INSERT INTO foo VALUES ('key', 'A'); --- inserts (current_time(), "key", "key", "A") +-- inserts (ROWTIME:=current_time(), KEY_COL:="key", COL_A:="A") INSERT INTO foo (KEY_COL, COL_A) VALUES ('key', 'A'); --- inserts (current_time(), "key", "key", null) +-- inserts (ROWTIME:=current_time(), KEY_COL:="key", COL_A:=null) INSERT INTO foo (KEY_COL) VALUES ('key'); ``` -The values are serialized by using the `value_format` specified in the -original `CREATE` statement. The key is always serialized as a String. +The values are serialized by using the format(s) specified in the original +`CREATE` statement. diff --git a/docs/developer-guide/ksqldb-reference/operators.md b/docs/developer-guide/ksqldb-reference/operators.md index 573ab96141e0..ed957414985c 100644 --- a/docs/developer-guide/ksqldb-reference/operators.md +++ b/docs/developer-guide/ksqldb-reference/operators.md @@ -22,12 +22,12 @@ the following table: ```sql CREATE TABLE USERS ( - USERID BIGINT + USERID BIGINT PRIMARY KEY, FIRST_NAME STRING, LAST_NAME STRING, NICKNAMES ARRAY, - ADDRESS STRUCT -) WITH (KAFKA_TOPIC='users', VALUE_FORMAT='AVRO', KEY='USERID'); + ADDRESS STRUCT +) WITH (KAFKA_TOPIC='users', VALUE_FORMAT='AVRO'); ``` Arithmetic @@ -37,7 +37,7 @@ The usual arithmetic operators (`+,-,/,*,%`) may be applied to numeric types, like INT, BIGINT, and DOUBLE: ```sql -SELECT LEN(FIRST_NAME) + LEN(LAST_NAME) AS NAME_LENGTH FROM USERS EMIT CHANGES; +SELECT USERID, LEN(FIRST_NAME) + LEN(LAST_NAME) AS NAME_LENGTH FROM USERS EMIT CHANGES; ``` Concatenation @@ -47,14 +47,15 @@ The concatenation operator (`+,||`) can be used to concatenate STRING values. ```sql -SELECT FIRST_NAME + LAST_NAME AS FULL_NAME FROM USERS EMIT CHANGES; +SELECT USERID, FIRST_NAME + LAST_NAME AS FULL_NAME FROM USERS EMIT CHANGES; ``` You can use the `+` operator for multi-part concatenation, for example: ```sql -SELECT TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') + +SELECT USERID, + TIMESTAMPTOSTRING(ROWTIME, 'yyyy-MM-dd HH:mm:ss') + ': :heavy_exclamation_mark: On ' + HOST + ' there were ' + @@ -72,7 +73,7 @@ The source dereference operator (`.`) can be used to specify columns by dereferencing the source stream or table. ```sql -SELECT USERS.FIRST_NAME FROM USERS EMIT CHANGES; +SELECT USERID, USERS.FIRST_NAME FROM USERS EMIT CHANGES; ``` Subscript @@ -82,7 +83,7 @@ The subscript operator (`[subscript_expr]`) is used to reference the value at an array index or a map key. ```sql -SELECT NICKNAMES[0] FROM USERS EMIT CHANGES; +SELECT USERID, NICKNAMES[0] FROM USERS EMIT CHANGES; ``` STRUCT dereference @@ -92,16 +93,12 @@ Access nested data by declaring a STRUCT and using the dereference operator (`->`) to access its fields: ```sql -CREATE STREAM orders ( - orderId BIGINT, - address STRUCT) WITH (...); - -SELECT address->street, address->zip FROM orders EMIT CHANGES; +SELECT USERID, ADDRESS->STREET, ADDRESS->HOUSE_NUM FROM USERS EMIT CHANGES; ``` -Combine -\> with . when using aliases: +Combine `->` with `.` when using aliases: ```sql -SELECT orders.address->street, o.address->zip FROM orders o EMIT CHANGES; +SELECT USERID, USERS.ADDRESS->STREET, U.ADDRESS->STREET FROM USERS U EMIT CHANGES; ``` diff --git a/docs/developer-guide/query-with-arrays-and-maps.md b/docs/developer-guide/query-with-arrays-and-maps.md index 1fc2e8ec6fb2..dc70fa8a78a1 100644 --- a/docs/developer-guide/query-with-arrays-and-maps.md +++ b/docs/developer-guide/query-with-arrays-and-maps.md @@ -32,15 +32,14 @@ array and map fields are defined in the `interests ARRAY` and ```sql CREATE TABLE users - (registertime BIGINT, - userid VARCHAR, + (userid VARCHAR PRIMARY KEY, + registertime BIGINT, gender VARCHAR, regionid VARCHAR, interests ARRAY, contactinfo MAP) WITH (KAFKA_TOPIC = 'users', - VALUE_FORMAT='JSON', - KEY = 'userid'); + VALUE_FORMAT='JSON'); ``` Your output should resemble: diff --git a/docs/developer-guide/syntax-reference.md b/docs/developer-guide/syntax-reference.md index d80e4c4a81c4..0504eb12c382 100644 --- a/docs/developer-guide/syntax-reference.md +++ b/docs/developer-guide/syntax-reference.md @@ -74,7 +74,7 @@ encapsulate a street address and a postal code: ```sql CREATE STREAM orders ( - ROWKEY BIGINT KEY, + ID BIGINT KEY, address STRUCT) WITH (...); ``` @@ -394,7 +394,7 @@ Use backtick characters to reference the columns: ```sql -- Enclose unparseable column names with backticks: -CREATE STREAM s1 (ROWKEY STRING KEY, `@id` integer, `col.val` string) … +CREATE STREAM s1 (K STRING KEY, `@id` integer, `col.val` string) … ``` Also, you can use backtick characters for the names of sources, like streams @@ -430,60 +430,41 @@ Key Requirements The `CREATE STREAM` and `CREATE TABLE` statements, which read data from a Kafka topic into a stream or table, allow you to specify a -field/column in the Kafka message value that corresponds to the Kafka -message key by setting the `KEY` property of the `WITH` clause. +`KEY` or `PRIMARY KEY` column, respectively, to represent the data in the Kafka message key. Example: ```sql -CREATE TABLE users (rowkey INT PRIMARY KEY, registertime BIGINT, gender VARCHAR, regionid VARCHAR, userid INT) - WITH (KAFKA_TOPIC='users', VALUE_FORMAT='JSON', KEY = 'userid'); +CREATE TABLE users (userId INT PRIMARY KEY, registertime BIGINT, gender VARCHAR, regionid VARCHAR) + WITH (KAFKA_TOPIC='users', VALUE_FORMAT='JSON'); ``` -The `KEY` property is optional. ksqlDB uses it as an optimization hint to -determine if repartitioning can be avoided when performing aggregations -and joins. +While tables require a `PRIMARY KEY`, the `KEY` column of a stream is optional. -The type of the column named in the `KEY` property must match the type of the `ROWKEY` column. +Joins involving tables can be joined to the table on the `PRIMARY KEY` column. Joins involving +streams have no such limitation. Stream joins on any expression other than the stream's `KEY` +column will require an internal repartition, joins on the stream's `KEY` column do not. !!! important - Don't set the KEY property, unless you have validated that your - stream doesn't need to be re-partitioned for future joins. If you set - the KEY property, you will need to re-partition explicitly if your - record key doesn't meet partitioning requirements. For more - information, see - [Partition Data to Enable Joins](joins/partition-data.md). - -In either case, when setting `KEY` you must be sure that *both* of the -following conditions are true: - -1. For every record, the contents of the Kafka message key must be the - same as the contents of the column set in `KEY` (which is derived - from a field in the Kafka message value). -2. `KEY` must be set to a value column with the same SQL type as the key column. - -If these conditions aren't met, then the results of aggregations and -joins may be incorrect. However, if your data doesn't meet these -requirements, you can still use ksqlDB with a few extra steps. The -following section explains how. - -Table-table joins can be joined only on the `KEY` field, and one-to-many -(1:N) joins aren't supported. + Be aware that Kafka guarantees the relative order of any two messages from + one source partition only if they are also both in the same partition + after the repartition. Otherwise, Kafka is likely to interleave messages. + The use case will determine if these ordering guarantees are acceptable. ### What To Do If Your Key Is Not Set or Is In A Different Format ### Streams -For streams, just leave out the `KEY` property from the `WITH` clause. +For streams, just leave out the `KEY` column from the column list. ksqlDB will take care of repartitioning the stream for you using the value(s) from the `GROUP BY` columns for aggregates, and the join -predicate for joins. +criteria for joins. ### Tables -For tables, you can still use ksqlDB if the message key is not also -present in the Kafka message value or if it is not in the required -format as long as *one* of the following statements is true: +For tables, you can still use ksqlDB if the message key is not set or if it is +not in the required format as long as the key can be rebuilt from the value data +and *one* of the following statements is true: - The message key is a [unary function](https://en.wikipedia.org/wiki/Unary_function) of the value @@ -491,16 +472,17 @@ format as long as *one* of the following statements is true: - It is ok for the messages in the topic to be re-ordered before being inserted into the table. -First create a stream to have ksqlDB write the message key, and then +First create a stream which we'll then use to have ksqlDB write the message key, and then declare the table on the output topic of this stream: Example: - Goal: You want to create a table from a topic, which is keyed by userid of type INT. -- Problem: The required key is present as a field/column (aptly named - `userid`) in the message value, but the actual message key in {{ site.ak }} is - not set or has some other value or format. +- Problem: The required key is present as a column (aptly named + `userid`) in the message value as is a string containg an integer, + but the actual message key in {{ site.ak }} is not set or has some + other value or format. ```sql -- Create a stream on the original topic @@ -508,38 +490,39 @@ CREATE STREAM users_with_wrong_key (userid INT, username VARCHAR, email VARCHAR) WITH (KAFKA_TOPIC='users', VALUE_FORMAT='JSON'); -- Derive a new stream with the required key changes. --- 1) The CAST statement converts the key to the required format. +-- 1) The CAST statement converts userId to the required SQL type. -- 2) The PARTITION BY clause re-partitions the stream based on the new, converted key. --- 3) The SELECT clause selects the required value columns, (key columns are implicitly included). --- The resulting schema will be: ROWKEY INT, USERNAME STRING, EMAIL STRING --- the userId will be stored in ROWKEY. +-- 3) The SELECT clause selects the required value columns, all in this case. +-- The resulting schema will be: KSQL_COL_0 INT KEY, USERNAME STRING, EMAIL STRING. +-- Note: the system generated KSQL_COL_0 column name can be replaced via an alias in the projection +-- however, this is not necessary in this instance as we do not intend to use this stream as a +-- source for other queries. CREATE STREAM users_with_proper_key WITH(KAFKA_TOPIC='users-with-proper-key') AS - SELECT username, email + SELECT * FROM users_with_wrong_key - PARTITION BY userid + PARTITION BY CAST(userid AS BIGINT) EMIT CHANGES; -- Now you can create the table on the properly keyed stream. -CREATE TABLE users_table (ROWKEY INT PRIMARY KEY, username VARCHAR, email VARCHAR) +CREATE TABLE users_table (userId INT PRIMARY KEY, username VARCHAR, email VARCHAR) WITH (KAFKA_TOPIC='users-with-proper-key', VALUE_FORMAT='JSON'); -- Or, if you prefer, you can keep userId in the value of the repartitioned data --- This enables using the more descriptive `userId` rather than ROWTIME. +-- by using the AS_VALUE function: +-- The resulting schema will be: userId INT KEY, USERNAME STRING, EMAIL STRING, VUSERID INT CREATE STREAM users_with_proper_key_and_user_id WITH(KAFKA_TOPIC='users_with_proper_key_and_user_id') AS - SELECT * + SELECT userId, username, email, AS_VALUE(userId) as vUserId FROM users_with_wrong_key PARTITION BY userid EMIT CHANGES; -- Now you can create the table on the properly keyed stream. --- queries against the table can use ROWKEY and userid interchangeably -CREATE TABLE users_table_2 (ROWKEY INT PRIMARY KEY, userid KEY, username VARCHAR, email VARCHAR) +CREATE TABLE users_table_2 (userId INT PRIMARY KEY, username VARCHAR, email VARCHAR, vUserId INT) WITH (KAFKA_TOPIC='users_with_proper_key_and_user_id', - VALUE_FORMAT='JSON', - KEY='userid'); + VALUE_FORMAT='JSON'); ``` For more information, see diff --git a/docs/developer-guide/test-and-debug/generate-custom-test-data.md b/docs/developer-guide/test-and-debug/generate-custom-test-data.md index 9807dfdd46c3..5f3715e7e062 100644 --- a/docs/developer-guide/test-and-debug/generate-custom-test-data.md +++ b/docs/developer-guide/test-and-debug/generate-custom-test-data.md @@ -162,14 +162,13 @@ In the ksqlDB CLI or in {{ site.c3short }}, register a table on ```sql CREATE TABLE users_original ( + userid VARCHAR PRIMARY KEY, registertime BIGINT, gender VARCHAR, - regionid VARCHAR, - userid VARCHAR) + regionid VARCHAR) WITH ( kafka_topic='users_kafka_topic_json', - value_format='JSON', - key = 'userid'); + value_format='JSON'); ``` Inspect the schema of the `users_original` table by using the DESCRIBE @@ -215,16 +214,15 @@ In the ksqlDB CLI or in {{ site.c3short }}, register a table on ```sql CREATE TABLE users_extended ( + userid VARCHAR PRIMARY KEY, registertime BIGINT, gender VARCHAR, regionid VARCHAR, - userid VARCHAR, interests ARRAY, contactInfo MAP) WITH ( kafka_topic='users_extended', - value_format='JSON', - key = 'userid'); + value_format='JSON'); ``` Inspect the schema of the `users_extended` table by using the DESCRIBE diff --git a/docs/operate-and-deploy/capacity-planning.md b/docs/operate-and-deploy/capacity-planning.md index a847cbf367ec..931c45d15de5 100644 --- a/docs/operate-and-deploy/capacity-planning.md +++ b/docs/operate-and-deploy/capacity-planning.md @@ -435,11 +435,11 @@ for the input data: ```sql CREATE STREAM pageviews_original - (viewtime BIGINT, userid VARCHAR, pageid VARCHAR, client_ip INT, url VARCHAR, duration BIGINT, from_url VARCHAR, analytics VARCHAR) - WITH (kafka_topic='pageviews', value_format=’JSON’, KEY=’userid’); + (userid VARCHAR KEY, viewtime BIGINT, pageid VARCHAR, client_ip INT, url VARCHAR, duration BIGINT, from_url VARCHAR, analytics VARCHAR) + WITH (kafka_topic='pageviews', value_format='JSON'); -CREATE TABLE users (registertime BIGINT, gender VARCHAR, city INT, country INT, userid VARCHAR, email VARCHAR) - WITH (kafka_topic='users', value_format='JSON', key = 'userid'); +CREATE TABLE users (userid VARCHAR PRIMARY KEY, registertime BIGINT, gender VARCHAR, city INT, country INT, email VARCHAR) + WITH (kafka_topic='users', value_format='JSON'); ``` The following assumptions are also made: @@ -509,7 +509,7 @@ up views by city: ```sql CREATE STREAM pageviews_meaningful_with_user_info WITH (PARTITIONS=64) AS - SELECT pv.viewtime, pv.userid, pv.pageid, pv.client_ip, pv.url, pv.duration, pv.from_url, u.city, u.country, u.gender, u.email + SELECT pv.userid, pv.viewtime, pv.pageid, pv.client_ip, pv.url, pv.duration, pv.from_url, u.city, u.country, u.gender, u.email FROM pageviews_meaningful pv LEFT JOIN users u ON pv.userid = u.userid EMIT CHANGES; diff --git a/docs/operate-and-deploy/installation/server-config/avro-schema.md b/docs/operate-and-deploy/installation/server-config/avro-schema.md index 51ee201626a9..b170388707ad 100644 --- a/docs/operate-and-deploy/installation/server-config/avro-schema.md +++ b/docs/operate-and-deploy/installation/server-config/avro-schema.md @@ -26,9 +26,13 @@ create new nested STRUCT data as the result of a query. For more info, see The following functionality is not supported: - Message *keys* in Avro, Protobuf, or JSON formats are not supported. Message - keys in ksqlDB are always interpreted as STRING format, which means ksqlDB - ignores schemas that have been registered for message keys, and the key is - read by using `StringDeserializer`. + keys in ksqlDB are always interpreted as KAFKA format, which means ksqlDB + ignores schemas that have been registered for message keys. + +While ksqlDB does not support loading the message key's schema from the {{ site.sr }}, +you can provide the key column definition within the `CREATE TABLE` or `CREATE STREAM` +statement. Where a `CREATE TABLE` or `CREATE STREAM` statement does not provide an +explicit key columns an implicit `ROWKEY STRING` column will be added. Configure ksqlDB for Avro, Protobuf, and JSON ============================================= @@ -91,14 +95,19 @@ substitute `PROTOBUF` or `JSON_SR` for `AVRO` in each statement. ### Create a new stream by reading Avro-formatted data The following statement shows how to create a new `pageviews` stream by -reading from a {{ site.ak }} topic that has Avro-formatted messages. +reading from a {{ site.ak }} topic that has Avro-formatted message values and +a Kafka-formatted INT message key. ```sql CREATE STREAM pageviews + (pageId INT KEY) WITH (KAFKA_TOPIC='pageviews-avro-topic', VALUE_FORMAT='AVRO'); ``` +If the key schema is not provided, the key of the data will be assumed to be +a single KAFKA serialized `STRING` named `ROWKEY`. + ### Create a new table by reading Avro-formatted data The following statement shows how to create a new `users` table by @@ -107,8 +116,7 @@ reading from a {{ site.ak }} topic that has Avro-formatted message values. ```sql CREATE TABLE users WITH (KAFKA_TOPIC='users-avro-topic', - VALUE_FORMAT='AVRO', - KEY='userid'); + VALUE_FORMAT='AVRO'); ``` In this example, you don't need to define any columns or data types in @@ -128,10 +136,9 @@ statement creates the `users` table with a 64-bit integer key and infers the value columns from the Avro schema. ```sql -CREATE TABLE users (ROWKEY BIGINT PRIMARY KEY) +CREATE TABLE users (userId BIGINT PRIMARY KEY) WITH (KAFKA_TOPIC='users-avro-topic', - VALUE_FORMAT='AVRO', - KEY='userid'); + VALUE_FORMAT='AVRO'); ``` ### Create a new stream with selected fields of Avro-formatted data @@ -146,7 +153,7 @@ the available fields in the Avro data. In this example, only the `viewtime` and `pageid` columns are picked. ```sql -CREATE STREAM pageviews_reduced (viewtime BIGINT, pageid VARCHAR) +CREATE STREAM pageviews_reduced (pageid VARCHAR KEY, viewtime BIGINT) WITH (KAFKA_TOPIC='pageviews-avro-topic', VALUE_FORMAT='AVRO'); ``` @@ -165,7 +172,7 @@ schema for the new `pageviews_avro` stream, and it registers the schema with {{ site.sr }}. ```sql -CREATE STREAM pageviews_json (viewtime BIGINT, userid VARCHAR, pageid VARCHAR) +CREATE STREAM pageviews_json (pageid VARCHAR KEY, viewtime BIGINT, userid VARCHAR) WITH (KAFKA_TOPIC='pageviews_kafka_topic_json', VALUE_FORMAT='JSON'); CREATE STREAM pageviews_avro diff --git a/docs/operate-and-deploy/installation/upgrading.md b/docs/operate-and-deploy/installation/upgrading.md index fc376d7ddb09..9c3c11014fd7 100644 --- a/docs/operate-and-deploy/installation/upgrading.md +++ b/docs/operate-and-deploy/installation/upgrading.md @@ -106,12 +106,75 @@ This will stop all processing and delete any internal topics in Kafka. ## Upgrade notes +### Upgrading from ksqlDB 0.9.0 to 0.10.0 + +!!! important + ksqlDB 0.10.0 is not backward compatible. Do not upgrade in-place. + +The following changes in SQL syntax and functionality may mean SQL statements that previously ran no longer run: + +### WITH(KEY) syntax removed. + +In previous versions all key columns where called `ROWKEY`. To enable a more user friendly name to be +used for the key column in queries it was possible to supply an alias for the key column in the WITH +clause, for example: + +```sql +CREATE TABLE INPUT (ROWKEY INT PRIMARY KEY, ID INT, V0 STRING) WITH (key='ID', ...); +``` + +With the above query the `ID` column can be used as an alias for `ROWKEY`. This approach required +the Kafka message value to contain an exact copy of the key. + +[KLIP-24](https://github.com/confluentinc/ksql/blob/master/design-proposals/klip-24-key-column-semantics-in-queries.md) +removed the restriction that key columns must be named `ROWKEY`, negating the need for the `WITH(KEY)` +syntax, which has been removed, also removing the requirement for the Kafka message value to contain +an exact copy of the key. + +Update your queries by removing the `KEY` fro the `WITH` clause and appropriately naming your +`KEY` and `PRIMARY KEY` columns. For example, the above CREATE TABLE statement can now be rewritten +as: + +```sql +CREATE TABLE INPUT (ID INT PRIMARY KEY, V0 STRING) WITH (...); +``` + +Unless the value format is `DELIMITED`, which means the value columns are _order dependant_, so dropping +the `ID` value column would result in a deserialization error or the wrong values being loaded. If using +`DELIMITED`, consider rewriting as: + +```sql +CREATE TABLE INPUT (ID INT PRIMARY KEY, ignoreMe INT, V0 STRING) WITH (...); +``` + +### Upgrading from ksqlDB 0.7.0+ to 0.9.0 + +!!! important + ksqlDB 0.9.0 is not backward compatible. Do not upgrade in-place. + +The following changes in SQL syntax and functionality may mean SQL statements that previously ran no longer run: + +### Table PRIMARY KEYs + +Tables now use `PRIMARY KEY` to define their primary key column rather than `KEY`. +Update your `CREATE TABLE` statements as required. + +```sql +CREATE TABLE OUTPUT (ROWKEY INT KEY, V0 STRING, V1 DOUBLE) WITH (...); +``` + +Will need to be updated to: + +```sql +CREATE TABLE OUTPUT (ROWKEY INT PRIMARY KEY, V0 STRING, V1 DOUBLE) WITH (...); +``` + ### Upgrading from ksqlDB 0.6.0 to 0.7.0 !!! important ksqlDB 0.7.0 is not backward compatible. Do not upgrade in-place. -The following changes in SQL syntax and functionality may mean SQL statements that previously ran not longer run: +The following changes in SQL syntax and functionality may mean SQL statements that previously ran no longer run: ### `PARTITION BY` and `GROUP BY` result schema changes: diff --git a/docs/tutorials/basics-docker.md b/docs/tutorials/basics-docker.md index 6f6b7e23842a..2237a9751744 100644 --- a/docs/tutorials/basics-docker.md +++ b/docs/tutorials/basics-docker.md @@ -260,7 +260,7 @@ Create a table, named `users_original`, from the `users` topic, specifying the `value_format` of `AVRO`. ```sql -CREATE TABLE users_original WITH (kafka_topic='users', value_format='AVRO', key = 'userid'); +CREATE TABLE users_original WITH (kafka_topic='users', value_format='AVRO'); ``` Your output should resemble: @@ -285,15 +285,6 @@ Your output should resemble: column in the statement, allowing you to specify the key column type. For example: `CREATE TABLE users_original (ROWKEY INT PRIMARY KEY) WITH (...);` -!!! note - The data generated has the same value in the {{ site.ak }} record's key - as the `userId` field in the value. Specifying `key='userId'` - in the WITH clause above lets ksqlDB know this. ksqlDB uses this information - to allow joins against the table to use the more - descriptive `userId` column name, rather than `ROWKEY`. Joining - on either yields the same results. If your data doesn't - contain a copy of the key in the value, you can join on `ROWKEY`. - !!! tip You can run `DESCRIBE users_original;` to see the schema for the Table. @@ -481,12 +472,12 @@ SELECT * FROM pageviews_enriched emit changes; Your output should resemble: ``` -+------------+------------+------------+------------+------------+ -|ROWKEY |USERID |PAGEID |REGIONID |GENDER | -+------------+------------+------------+------------+------------+ -|User_5 |User_5 |Page_53 |Region_3 |FEMALE | -|User_2 |User_2 |Page_86 |Region_5 |OTHER | -|User_9 |User_9 |Page_75 |Region_1 |OTHER | ++------------+------------+------------+------------+ +|USERID |PAGEID |REGIONID |GENDER | ++------------+------------+------------+------------+ +|User_5 |Page_53 |Region_3 |FEMALE | +|User_2 |Page_86 |Region_5 |OTHER | +|User_9 |Page_75 |Region_1 |OTHER | ^CQuery terminated ``` @@ -585,14 +576,14 @@ SELECT * FROM pageviews_regions EMIT CHANGES LIMIT 5; Your output should resemble: ``` -+-----------------+---------------+---------------+---------------+---------------+---------------+ -|ROWKEY |WINDOWSTART |WINDOWEND |GENDER |REGIONID |NUMUSERS | -+-----------------+---------------+---------------+---------------+---------------+---------------+ -|OTHER|+|Region_9 |1581080490000 |1581080520000 |OTHER |Region_9 |1 | -|OTHER|+|Region_5 |1581080490000 |1581080520000 |OTHER |Region_5 |2 | -|MALE|+|Region_7 |1581080490000 |1581080520000 |MALE |Region_7 |4 | -|FEMALE|+|Region_1|1581080490000 |1581080520000 |FEMALE |Region_1 |2 | -|MALE|+|Region_2 |1581080490000 |1581080520000 |MALE |Region_2 |3 | ++-----------------+---------------+---------------+---------------+ +|KSQL_COL_0 |WINDOWSTART |WINDOWEND |NUMUSERS | ++-----------------+---------------+---------------+---------------+ +|OTHER|+|Region_9 |1581080490000 |1581080520000 |1 | +|OTHER|+|Region_5 |1581080490000 |1581080520000 |2 | +|MALE|+|Region_7 |1581080490000 |1581080520000 |4 | +|FEMALE|+|Region_1|1581080490000 |1581080520000 |2 | +|MALE|+|Region_2 |1581080490000 |1581080520000 |3 | Limit Reached Query terminated ``` @@ -603,6 +594,12 @@ Query terminated per 30 second _window_. ksqlDB automatically adds these system columns for windowed results. +!!! note + The grouping columns `gender` and `regionid` are currently stored combined + in a single key column. This will be rectified once multiple key columns + are supported. Until then, then columns are combined and the resulting column + if given a system generated column name. + ### 8. View query results using a pull query When a CREATE TABLE statement contains a GROUP BY clause, ksqlDB builds an internal @@ -617,27 +614,29 @@ Pull queries do not have the `EMIT CHANGES` clause. View all of the windows and user counts that are available for a specific gender and region by using a pull query: ```sql -SELECT * FROM pageviews_regions WHERE ROWKEY='OTHER|+|Region_9'; +SELECT * FROM pageviews_regions WHERE KSQL_COL_0='OTHER|+|Region_9'; ``` Your output should resemble: ``` -+------------------+------------------+------------------+------------------+------------------+------------------+ -|ROWKEY |WINDOWSTART |WINDOWEND |GENDER |REGIONID |NUMUSERS | -+------------------+------------------+------------------+------------------+------------------+------------------+ -|OTHER|+|Region_9 |1581080490000 |1581080520000 |OTHER |Region_9 |1 | -|OTHER|+|Region_9 |1581080550000 |1581080580000 |OTHER |Region_9 |4 | -|OTHER|+|Region_9 |1581080580000 |1581080610000 |OTHER |Region_9 |4 | -|OTHER|+|Region_9 |1581080610000 |1581080640000 |OTHER |Region_9 |3 | -|OTHER|+|Region_9 |1581080640000 |1581080670000 |OTHER |Region_9 |6 | ++------------------+------------------+------------------+------------------+ +|KSQL_COL_0 |WINDOWSTART |WINDOWEND |NUMUSERS | ++------------------+------------------+------------------+------------------+ +|OTHER|+|Region_9 |1581080490000 |1581080520000 |1 | +|OTHER|+|Region_9 |1581080550000 |1581080580000 |4 | +|OTHER|+|Region_9 |1581080580000 |1581080610000 |4 | +|OTHER|+|Region_9 |1581080610000 |1581080640000 |3 | +|OTHER|+|Region_9 |1581080640000 |1581080670000 |6 | ... ``` Pull queries on windowed tables such as `pageviews_regions` also support querying a single window's result: ```sql -SELECT NUMUSERS FROM pageviews_regions WHERE ROWKEY='OTHER|+|Region_9' AND WINDOWSTART=1581080550000; +SELECT NUMUSERS FROM pageviews_regions + WHERE KSQL_COL_0='OTHER|+|Region_9' + AND WINDOWSTART=1581080550000; ``` !!! important @@ -657,7 +656,9 @@ Query terminated Or querying a range of windows: ```sql -SELECT WINDOWSTART, WINDOWEND, NUMUSERS FROM pageviews_regions WHERE ROWKEY='OTHER|+|Region_9' AND 1581080550000 <= WINDOWSTART AND WINDOWSTART <= 1581080610000; +SELECT WINDOWSTART, WINDOWEND, NUMUSERS FROM pageviews_regions + WHERE KSQL_COL_0='OTHER|+|Region_9' + AND 1581080550000 <= WINDOWSTART AND WINDOWSTART <= 1581080610000; ``` !!! important @@ -711,7 +712,6 @@ Your output should resemble: ``` Name : PAGEVIEWS_REGIONS Type : TABLE -Key field : Timestamp field : Not set - using Key format : KAFKA Value format : AVRO @@ -725,13 +725,11 @@ WINDOW TUMBLING ( SIZE 30 SECONDS ) GROUP BY PAGEVIEWS_ENRICHED.GENDER, PAGEVIEWS_ENRICHED.REGIONID EMIT CHANGES; - Field | Type --------------------------------------------------------------- - ROWKEY | VARCHAR(STRING) (key) (Window type: TUMBLING) - GENDER | VARCHAR(STRING) - REGIONID | VARCHAR(STRING) - NUMUSERS | BIGINT --------------------------------------------------------------- + Field | Type +---------------------------------------------------------------- + KSQL_COL_0 | VARCHAR(STRING) (key) (Window type: TUMBLING) + NUMUSERS | BIGINT +---------------------------------------------------------------- Queries that write from this TABLE ----------------------------------- @@ -774,14 +772,13 @@ From the ksqlDB command prompt, register the a stream on the `orders` topic: ```sql CREATE STREAM ORDERS ( - ROWKEY INT KEY, + ORDERID INT KEY, ORDERTIME BIGINT, - ORDERID INT, - ITEMID STRING, + ITEMID STRING, ORDERUNITS DOUBLE, ADDRESS STRUCT ) - WITH (KAFKA_TOPIC='orders', VALUE_FORMAT='json', key='orderid'); + WITH (KAFKA_TOPIC='orders', VALUE_FORMAT='json'); ``` Your output should resemble: @@ -808,9 +805,8 @@ Your output should resemble: Name : ORDERS Field | Type ---------------------------------------------------------------------------------- - ROWKEY | INT (key) + ORDERID | INT (key) ORDERTIME | BIGINT - ORDERID | INTEGER ITEMID | VARCHAR(STRING) ORDERUNITS | DOUBLE ADDRESS | STRUCT @@ -852,14 +848,13 @@ see shipment information alongside the order. ### 1. Create two streams -In the ksqlDB CLI create two new streams. Both streams will store their -order ID in ROWKEY: +In the ksqlDB CLI create two new streams: ```sql -CREATE STREAM NEW_ORDERS (ROWKEY INT KEY, TOTAL_AMOUNT DOUBLE, CUSTOMER_NAME VARCHAR) +CREATE STREAM NEW_ORDERS (ORDER_ID INT KEY, TOTAL_AMOUNT DOUBLE, CUSTOMER_NAME VARCHAR) WITH (KAFKA_TOPIC='new_orders', VALUE_FORMAT='JSON', PARTITIONS=2); -CREATE STREAM SHIPMENTS (ROWKEY INT KEY, SHIPMENT_ID INT, WAREHOUSE VARCHAR) +CREATE STREAM SHIPMENTS (ORDER_ID INT KEY, SHIPMENT_ID INT, WAREHOUSE VARCHAR) WITH (KAFKA_TOPIC='shipments', VALUE_FORMAT='JSON', PARTITIONS=2); ``` @@ -883,7 +878,7 @@ Populate the streams with some sample data by using the INSERT VALUES statement: ```sql -- Insert values in NEW_ORDERS: -- insert supplying the list of columns to insert: -INSERT INTO NEW_ORDERS (ROWKEY, CUSTOMER_NAME, TOTAL_AMOUNT) +INSERT INTO NEW_ORDERS (ORDER_ID, CUSTOMER_NAME, TOTAL_AMOUNT) VALUES (1, 'Bob Smith', 10.50); -- shorthand version can be used when inserting values for all columns, (except ROWTIME), in column order: @@ -922,7 +917,7 @@ Your output should resemble: ``` +-------------------------+-------------------------+-------------------------+-------------------------+ -|ROWTIME |ROWKEY |TOTAL_AMOUNT |CUSTOMER_NAME | +|ROWTIME |ORDER_ID |TOTAL_AMOUNT |CUSTOMER_NAME | +-------------------------+-------------------------+-------------------------+-------------------------+ |1581083057609 |1 |10.5 |Bob Smith | |1581083178418 |2 |3.32 |Sarah Black | @@ -941,7 +936,7 @@ Your output should resemble: ``` +-------------------------+-------------------------+-------------------------+ -|ROWKEY |SHIPMENT_ID |WAREHOUSE | +|ORDER_ID |SHIPMENT_ID |WAREHOUSE | +-------------------------+-------------------------+-------------------------+ |1 |42 |Nashville | |3 |43 |Palo Alto | @@ -955,12 +950,12 @@ Run the following query, which will show orders with associated shipments, based on a join window of 1 hour. ```sql -SELECT O.ROWKEY AS ORDER_ID, O.TOTAL_AMOUNT, O.CUSTOMER_NAME, +SELECT O.ORDER_ID AS ORDER_ID, O.TOTAL_AMOUNT, O.CUSTOMER_NAME, S.SHIPMENT_ID, S.WAREHOUSE FROM NEW_ORDERS O INNER JOIN SHIPMENTS S WITHIN 1 HOURS - ON O.ROWKEY = S.ROWKEY + ON O.ORDER_ID = S.ORDER_ID EMIT CHANGES; ``` @@ -1020,23 +1015,19 @@ enriched with data about the size of the warehouse from another. ### 1. Register two tables -In the KSQL CLI, register both topics as KSQL tables. Note, in this example -the warehouse id is stored both in the key and in the WAREHOUSE_ID field -in the value: +In the KSQL CLI, register both topics as KSQL tables. ```sql CREATE TABLE WAREHOUSE_LOCATION - (ROWKEY INT PRIMARY KEY, WAREHOUSE_ID INT, CITY VARCHAR, COUNTRY VARCHAR) + (WAREHOUSE_ID INT PRIMARY KEY, CITY VARCHAR, COUNTRY VARCHAR) WITH (KAFKA_TOPIC='warehouse_location', VALUE_FORMAT='JSON', - KEY='WAREHOUSE_ID', PARTITIONS=2); CREATE TABLE WAREHOUSE_SIZE - (ROWKEY INT PRIMARY KEY, WAREHOUSE_ID INT, SQUARE_FOOTAGE DOUBLE) + (WAREHOUSE_ID INT PRIMARY KEY, SQUARE_FOOTAGE DOUBLE) WITH (KAFKA_TOPIC='warehouse_size', VALUE_FORMAT='JSON', - KEY='WAREHOUSE_ID', PARTITIONS=2); ``` @@ -1054,7 +1045,6 @@ After both `CREATE TABLE` statements, your output should resemble: In the KSQL CLI, insert sample data into the tables: ```sql --- note: ksqlDB will automatically populate ROWKEY with the same value as WAREHOUSE_ID: INSERT INTO WAREHOUSE_LOCATION (WAREHOUSE_ID, CITY, COUNTRY) VALUES (1, 'Leeds', 'UK'); INSERT INTO WAREHOUSE_LOCATION (WAREHOUSE_ID, CITY, COUNTRY) VALUES (2, 'Sheffield', 'UK'); INSERT INTO WAREHOUSE_LOCATION (WAREHOUSE_ID, CITY, COUNTRY) VALUES (3, 'Berlin', 'Germany'); @@ -1064,51 +1054,7 @@ INSERT INTO WAREHOUSE_SIZE (WAREHOUSE_ID, SQUARE_FOOTAGE) VALUES (2, 42000); INSERT INTO WAREHOUSE_SIZE (WAREHOUSE_ID, SQUARE_FOOTAGE) VALUES (3, 94000); ``` -### 3. Examine tables for keys - -Check both tables that the message key (`ROWKEY`) matches the declared -key (`WAREHOUSE_ID`). The output should show that they are equal. If -they were not, the join won't succeed or behave as expected. - -Inspect the WAREHOUSE_LOCATION table: - -```sql -SELECT ROWKEY, WAREHOUSE_ID FROM WAREHOUSE_LOCATION EMIT CHANGES LIMIT 3; -``` - -Your output should resemble: - -``` -+---------------------------------------+---------------------------------------+ -|ROWKEY |WAREHOUSE_ID | -+---------------------------------------+---------------------------------------+ -|2 |2 | -|1 |1 | -|3 |3 | -Limit Reached -Query terminated -``` - -Inspect the WAREHOUSE_SIZE table: - -```sql -SELECT ROWKEY, WAREHOUSE_ID FROM WAREHOUSE_SIZE EMIT CHANGES LIMIT 3; -``` - -Your output should resemble: - -``` -+---------------------------------------+---------------------------------------+ -|ROWKEY |WAREHOUSE_ID | -+---------------------------------------+---------------------------------------+ -|2 |2 | -|1 |1 | -|3 |3 | -Limit Reached -Query terminated -``` - -### 4. Join the tables +### 3. Join the tables Now join the two tables: @@ -1175,10 +1121,9 @@ In ksqlDB, register the source topic for each: ```sql CREATE STREAM ORDERS_SRC_LOCAL ( - ROWKEY INT KEY, + ORDERID INT KEY, ORDERTIME BIGINT, - ORDERID INT, - ITEMID STRING, + ITEMID STRING, ORDERUNITS DOUBLE, ADDRESS STRUCT ) @@ -1186,10 +1131,9 @@ CREATE STREAM ORDERS_SRC_LOCAL CREATE STREAM ORDERS_SRC_3RDPARTY ( - ROWKEY INT KEY, + ORDERID INT KEY, ORDERTIME BIGINT, - ORDERID INT, - ITEMID STRING, + ITEMID STRING, ORDERUNITS DOUBLE, ADDRESS STRUCT ) @@ -1239,11 +1183,10 @@ Your output should resemble: Name : ALL_ORDERS Field | Type ---------------------------------------------------------------------------------- - ROWKEY | INTEGER (key) + ORDERID | INTEGER (key) SRC | VARCHAR(STRING) ORDERTIME | BIGINT - ORDERID | INTEGER - ITEMID | VARCHAR(STRING) + ITEMID | VARCHAR(STRING) ORDERUNITS | DOUBLE ADDRESS | STRUCT ---------------------------------------------------------------------------------- @@ -1279,12 +1222,12 @@ SELECT * FROM ALL_ORDERS EMIT CHANGES; Your output should resemble: ``` -+----------+-----------+--------------+----------+-------------+----------------------+---------------------------------------------+ -|ROWKEY |SRC |ORDERTIME |ORDERID |ITEMID |ORDERUNITS |ADDRESS | -+----------+-----------+--------------+----------+-------------+----------------------+---------------------------------------------+ -|510 |3RD PARTY |1503198352036 |510 |Item_643 |1.653210222047296 |{CITY=City_94, STATE=State_72, ZIPCODE=61274}| -|546 |LOCAL |1498476865306 |546 |Item_234 |9.284691223615178 |{CITY=City_44, STATE=State_29, ZIPCODE=84678}| -|511 |3RD PARTY |1489945722538 |511 |Item_264 |8.213163488516212 |{CITY=City_36, STATE=State_13, ZIPCODE=44821}| ++----------+-----------+--------------+-------------+----------------------+---------------------------------------------+ +|ORDERID |SRC |ORDERTIME |ITEMID |ORDERUNITS |ADDRESS | ++----------+-----------+--------------+-------------+----------------------+---------------------------------------------+ +|510 |3RD PARTY |1503198352036 |Item_643 |1.653210222047296 |{CITY=City_94, STATE=State_72, ZIPCODE=61274}| +|546 |LOCAL |1498476865306 |Item_234 |9.284691223615178 |{CITY=City_44, STATE=State_29, ZIPCODE=84678}| +|511 |3RD PARTY |1489945722538 |Item_264 |8.213163488516212 |{CITY=City_36, STATE=State_13, ZIPCODE=44821}| … ``` @@ -1408,10 +1351,10 @@ docker-compose exec kafka kafka-console-producer \ Your data input should resemble: ``` -key1:{"id":"key1","col1":"v1","col2":"v2","col3":"v3"} -key2:{"id":"key2","col1":"v4","col2":"v5","col3":"v6"} -key3:{"id":"key3","col1":"v7","col2":"v8","col3":"v9"} -key1:{"id":"key1","col1":"v10","col2":"v11","col3":"v12"} +key1:{"col1":"v1","col2":"v2","col3":"v3"} +key2:{"col1":"v4","col2":"v5","col3":"v6"} +key3:{"col1":"v7","col2":"v8","col3":"v9"} +key1:{"col1":"v10","col2":"v11","col3":"v12"} ``` You can also use the `kafkacat` command line tool: @@ -1423,9 +1366,9 @@ docker run --interactive --rm --network tutorials_default \ -t warehouse_size \ -K: \ -P <, contactinfo map) WITH (KAFKA_TOPIC='users', - VALUE_FORMAT='JSON', - KEY = 'userid'); + VALUE_FORMAT='JSON'); ``` -Note that specifying KEY is required in table declaration, see +Note that specifying the table's PRIMARY KEY is required in table declaration, see [Key Requirements](../developer-guide/syntax-reference.md#key-requirements). Working with streams and tables diff --git a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index 649944cb113c..ab991054d28b 100644 --- a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -451,7 +451,6 @@ private void printWarnings(final KsqlEntity entity) { private static String formatFieldType( final FieldInfo field, final Optional windowType, - final String keyField, final boolean isTable ) { final FieldType possibleFieldType = field.getType().orElse(null); @@ -465,17 +464,12 @@ private static String formatFieldType( return String.format("%-16s %s%s", field.getSchema().toTypeString(), keyType, wt); } - if (keyField != null && keyField.contains("." + field.getName())) { - return String.format("%-16s %s", field.getSchema().toTypeString(), "(key)"); - } - return field.getSchema().toTypeString(); } private void printSchema( final Optional windowType, final List fields, - final String keyField, final boolean isTable ) { final Table.Builder tableBuilder = new Table.Builder(); @@ -483,7 +477,7 @@ private void printSchema( tableBuilder.withColumnHeaders("Field", "Type"); fields.forEach(f -> tableBuilder.withRow( f.getName(), - formatFieldType(f, windowType, keyField, isTable) + formatFieldType(f, windowType, isTable) )); tableBuilder.build().print(this); } @@ -494,7 +488,6 @@ private void printTopicInfo(final SourceDescription source) { ? "Not set - using " : source.getTimestamp(); - writer().println(String.format("%-20s : %s", "Key field", source.getKey())); writer().println(String.format("%-20s : %s", "Timestamp field", timestamp)); writer().println(String.format("%-20s : %s", "Key format", source.getKeyFormat())); writer().println(String.format("%-20s : %s", "Value format", source.getValueFormat())); @@ -587,7 +580,7 @@ private void printSourceDescription(final SourceDescription source) { writer().println(String.format("%-20s : %s", "Name", source.getName())); if (!source.isExtended()) { - printSchema(source.getWindowType(), source.getFields(), source.getKey(), isTable); + printSchema(source.getWindowType(), source.getFields(), isTable); writer().println( "For runtime statistics and query details run: DESCRIBE EXTENDED ;"); return; @@ -598,7 +591,7 @@ private void printSourceDescription(final SourceDescription source) { writer().println(String.format("%-20s : %s", "Statement", source.getStatement())); writer().println(""); - printSchema(source.getWindowType(), source.getFields(), source.getKey(), isTable); + printSchema(source.getWindowType(), source.getFields(), isTable); printQueries(source.getReadQueries(), source.getType(), "read"); @@ -666,7 +659,7 @@ private void printQueryDescription(final QueryDescription query) { query.getKsqlHostQueryStatus())); } writer().println(); - printSchema(query.getWindowType(), query.getFields(), "", false); + printSchema(query.getWindowType(), query.getFields(), false); printQuerySources(query); printQuerySinks(query); printExecutionPlan(query); diff --git a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/JLineReader.java b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/JLineReader.java index 209d2981b51d..c845628f7118 100644 --- a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/JLineReader.java +++ b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/JLineReader.java @@ -127,7 +127,7 @@ private static class KsqlExpander extends DefaultExpander { private static final String EXPANDED_CT = "CREATE TABLE t (field1 type1, field2 type2) " - + "WITH (KAFKA_TOPIC='topic-name', VALUE_FORMAT='json', KEY='field1');"; + + "WITH (KAFKA_TOPIC='topic-name', VALUE_FORMAT='json');"; private static final Map shortcuts = ImmutableMap.of( "cs", EXPANDED_CS, diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/CliTest.java b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/CliTest.java index 31ff9509ddeb..c8cf8801575b 100644 --- a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/CliTest.java +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/CliTest.java @@ -38,6 +38,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.common.utils.IntegrationTest; import io.confluent.ksql.GenericRow; import io.confluent.ksql.TestTerminal; @@ -79,7 +82,6 @@ import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -111,6 +113,7 @@ * Most tests in CliTest are end-to-end integration tests, so it may expect a long running time. */ @SuppressWarnings("SameParameterValue") +@SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"}) @RunWith(MockitoJUnitRunner.class) @Category(IntegrationTest.class) public class CliTest { @@ -149,7 +152,7 @@ public class CliTest { @Rule public final Timeout timeout = Timeout.builder() - .withTimeout(30, TimeUnit.SECONDS) + .withTimeout(1, TimeUnit.MINUTES) .withLookingForStuckThread(true) .build(); @@ -262,11 +265,10 @@ public static void classTearDown() { private void testCreateStreamAsSelect( final String selectQuery, final PhysicalSchema resultSchema, - final Map expectedResults + final Map expectedResults ) { final String queryString = "CREATE STREAM " + streamName + " AS " + selectQuery; - /* Start Stream Query */ assertRunCommand( queryString, anyOf( @@ -274,17 +276,16 @@ private void testCreateStreamAsSelect( isRow(is("Parsing statement")), isRow(is("Executing statement")))); - /* Assert Results */ - final Map results = TEST_HARNESS.verifyAvailableUniqueRows( - streamName, - expectedResults.size(), - FormatFactory.JSON, - resultSchema - ); - - dropStream(streamName); - - assertThat(results, equalTo(expectedResults)); + try { + TEST_HARNESS.verifyAvailableUniqueRows( + streamName, + is(expectedResults), + FormatFactory.JSON, + resultSchema + ); + } finally { + dropStream(streamName); + } } private static void runStatement(final String statement, final KsqlRestClient restClient) { @@ -401,12 +402,11 @@ public void shouldPrintTopicWithJsonValue() { // Then: assertThatEventually(() -> terminal.getOutputString(), containsString("Value format: JSON")); assertThatEventually(() -> terminal.getOutputString(), - containsString("Key format: KAFKA_BIGINT or KAFKA_DOUBLE")); + containsString("Key format: KAFKA_STRING")); assertThatEventually(() -> terminal.getOutputString(), containsString("," - + " key: 1, " + + " key: ORDER_1, " + "value: {" + "\"ORDERTIME\":1," - + "\"ORDERID\":\"ORDER_1\"," + "\"ITEMID\":\"ITEM_1\"," + "\"ORDERUNITS\":10.0," + "\"TIMESTAMP\":\"2018-01-01\"," @@ -426,7 +426,7 @@ public void shouldPrintTopicWithDelimitedValue() { assertThat(terminal.getOutputString(), containsString("Key format: KAFKA_STRING")); assertThat(terminal.getOutputString(), containsString(", key: , value: ")); assertThat(terminal.getOutputString(), - containsString(", key: ITEM_1, value: ITEM_1,home cinema")); + containsString(", key: ITEM_1, value: home cinema")); } @Test @@ -487,9 +487,8 @@ public void shouldPrintCorrectSchemaForDescribeStream() { assertRunCommand( "describe " + ORDER_DATA_PROVIDER.kstreamName() + ";", containsRows( - row("ROWKEY", "BIGINT (key)"), + row("ORDERID", "VARCHAR(STRING) (key)"), row("ORDERTIME", "BIGINT"), - row("ORDERID", "VARCHAR(STRING)"), row("ITEMID", "VARCHAR(STRING)"), row("ORDERUNITS", "DOUBLE"), row("TIMESTAMP", "VARCHAR(STRING)"), @@ -503,56 +502,43 @@ public void testPersistentSelectStar() { testCreateStreamAsSelect( "SELECT * FROM " + ORDER_DATA_PROVIDER.kstreamName() + ";", ORDER_DATA_PROVIDER.schema(), - ORDER_DATA_PROVIDER.data() + ORDER_DATA_PROVIDER.finalData() ); } @Test public void testSelectProject() { - final Map expectedResults = new HashMap<>(); - expectedResults.put(1L, genericRow( - "ITEM_1", - 10.0, - ImmutableList.of(100.0, 110.99, 90.0))); - - expectedResults.put(2L, genericRow( - "ITEM_2", - 20.0, - ImmutableList.of(10.0, 10.99, 9.0))); - - expectedResults.put(3L, genericRow( - "ITEM_3", - 30.0, - ImmutableList.of(10.0, 10.99, 91.0))); - - expectedResults.put(4L, genericRow( - "ITEM_4", - 40.0, - ImmutableList.of(10.0, 140.99, 94.0))); - - expectedResults.put(5L, genericRow( - "ITEM_5", - 50.0, - ImmutableList.of(160.0, 160.99, 98.0))); - - expectedResults.put(6L, genericRow( - "ITEM_6", - 60.0, - ImmutableList.of(1000.0, 1100.99, 900.0))); - - expectedResults.put(7L, genericRow( - "ITEM_7", - 70.0, - ImmutableList.of(1100.0, 1110.99, 190.0))); - - expectedResults.put(8L, genericRow( - "ITEM_8", - 80.0, - ImmutableList.of(1100.0, 1110.99, 970.0))); + final Map expectedResults = ImmutableMap + .builder() + .put("ORDER_1", genericRow( + "ITEM_1", + 10.0, + ImmutableList.of(100.0, 110.99, 90.0))) + .put("ORDER_2", genericRow( + "ITEM_2", + 20.0, + ImmutableList.of(10.0, 10.99, 9.0))) + .put("ORDER_3", genericRow( + "ITEM_3", + 30.0, + ImmutableList.of(10.0, 10.99, 91.0))) + .put("ORDER_4", genericRow( + "ITEM_4", + 40.0, + ImmutableList.of(10.0, 140.99, 94.0))) + .put("ORDER_5", genericRow( + "ITEM_5", + 50.0, + ImmutableList.of(160.0, 160.99, 98.0))) + .put("ORDER_6", genericRow( + "ITEM_8", + 80.0, + ImmutableList.of(1100.0, 1110.99, 970.0))) + .build(); final PhysicalSchema resultSchema = PhysicalSchema.from( LogicalSchema.builder() - .keyColumns(ORDER_DATA_PROVIDER.schema().logicalSchema().key()) + .keyColumn(ColumnName.of("ORDERID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ORDERUNITS"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("PRICEARRAY"), SqlTypes.array(SqlTypes.DOUBLE)) @@ -561,7 +547,8 @@ public void testSelectProject() { ); testCreateStreamAsSelect( - "SELECT ROWKEY, ITEMID, ORDERUNITS, PRICEARRAY FROM " + ORDER_DATA_PROVIDER.kstreamName() + ";", + "SELECT ORDERID, ITEMID, ORDERUNITS, PRICEARRAY " + + "FROM " + ORDER_DATA_PROVIDER.kstreamName() + ";", resultSchema, expectedResults ); @@ -569,22 +556,21 @@ public void testSelectProject() { @Test public void testSelectFilter() { - final Map expectedResults = new HashMap<>(); - final Map mapField = new HashMap<>(); - mapField.put("key1", 1.0); - mapField.put("key2", 2.0); - mapField.put("key3", 3.0); - expectedResults.put(8L, genericRow( - 8L, + final Map expectedResults = ImmutableMap.of( "ORDER_6", - "ITEM_8", - 80.0, - "2018-01-08", - ImmutableList.of(1100.0, 1110.99, 970.0), - mapField)); + genericRow( + 8L, + "ITEM_8", + 80.0, + "2018-01-08", + ImmutableList.of(1100.0, 1110.99, 970.0), + ImmutableMap.of("key1", 1.0, "key2", 2.0, "key3", 3.0) + ) + ); testCreateStreamAsSelect( - "SELECT * FROM " + ORDER_DATA_PROVIDER.kstreamName() + " WHERE ORDERUNITS > 20 AND ITEMID = 'ITEM_8';", + "SELECT * FROM " + ORDER_DATA_PROVIDER.kstreamName() + + " WHERE ORDERUNITS > 20 AND ITEMID = 'ITEM_8';", ORDER_DATA_PROVIDER.schema(), expectedResults ); @@ -592,18 +578,18 @@ public void testSelectFilter() { @Test public void testTransientSelect() { - final Map streamData = ORDER_DATA_PROVIDER.data(); - final List row1 = streamData.get(1L).values(); - final List row2 = streamData.get(2L).values(); - final List row3 = streamData.get(3L).values(); + final Multimap streamData = ORDER_DATA_PROVIDER.data(); + final List row1 = Iterables.getFirst(streamData.get("ORDER_1"), genericRow()).values(); + final List row2 = Iterables.getFirst(streamData.get("ORDER_2"), genericRow()).values(); + final List row3 = Iterables.getFirst(streamData.get("ORDER_3"), genericRow()).values(); selectWithLimit( "SELECT ORDERID, ITEMID FROM " + ORDER_DATA_PROVIDER.kstreamName() + " EMIT CHANGES", 3, containsRows( - row(row1.get(1).toString(), row1.get(2).toString()), - row(row2.get(1).toString(), row2.get(2).toString()), - row(row3.get(1).toString(), row3.get(2).toString()) + row("ORDER_1", row1.get(1).toString()), + row("ORDER_2", row2.get(1).toString()), + row("ORDER_3", row3.get(1).toString()) )); } @@ -655,18 +641,18 @@ public void shouldOutputPullQueryHeader() { @Test public void testTransientContinuousSelectStar() { - final Map streamData = ORDER_DATA_PROVIDER.data(); - final List row1 = streamData.get(1L).values(); - final List row2 = streamData.get(2L).values(); - final List row3 = streamData.get(3L).values(); + final Multimap streamData = ORDER_DATA_PROVIDER.data(); + final List row1 = Iterables.getFirst(streamData.get("ORDER_1"), genericRow()).values(); + final List row2 = Iterables.getFirst(streamData.get("ORDER_2"), genericRow()).values(); + final List row3 = Iterables.getFirst(streamData.get("ORDER_3"), genericRow()).values(); selectWithLimit( "SELECT * FROM " + ORDER_DATA_PROVIDER.kstreamName() + " EMIT CHANGES", 3, containsRows( - row(prependWithKey(row1)), - row(prependWithKey(row2)), - row(prependWithKey(row3)) + row(prependWithKey("ORDER_1", row1)), + row(prependWithKey("ORDER_2", row2)), + row(prependWithKey("ORDER_3", row3)) )); } @@ -676,7 +662,6 @@ public void shouldOutputPushQueryHeader() { run("SELECT * FROM " + ORDER_DATA_PROVIDER.kstreamName() + " EMIT CHANGES LIMIT 1;", localCli); // Then: (note that some of these are truncated because of header wrapping) - assertThat(terminal.getOutputString(), containsString("ROWKEY")); assertThat(terminal.getOutputString(), containsString("ITEMID")); assertThat(terminal.getOutputString(), containsString("ORDERID")); assertThat(terminal.getOutputString(), containsString("ORDERUNIT")); @@ -689,7 +674,7 @@ public void shouldOutputPushQueryHeader() { public void testSelectUDFs() { final String queryString = String.format( "SELECT " - + "ROWKEY, " + + "ORDERID, " + "ITEMID, " + "ORDERUNITS*10 AS Col1, " + "PRICEARRAY[1]+10 AS Col2, " @@ -702,7 +687,7 @@ public void testSelectUDFs() { final PhysicalSchema resultSchema = PhysicalSchema.from( LogicalSchema.builder() - .keyColumns(ORDER_DATA_PROVIDER.schema().logicalSchema().key()) + .keyColumn(ColumnName.of("ORDERID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) .valueColumn(ColumnName.of("COL1"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("COL2"), SqlTypes.DOUBLE) @@ -712,8 +697,10 @@ public void testSelectUDFs() { SerdeOption.none() ); - final Map expectedResults = new HashMap<>(); - expectedResults.put(8L, genericRow("ITEM_8", 800.0, 1110.0, 12.0, true)); + final Map expectedResults = ImmutableMap.of( + "ORDER_6", + genericRow("ITEM_8", 800.0, 1110.0, 12.0, true) + ); testCreateStreamAsSelect(queryString, resultSchema, expectedResults); } @@ -1179,10 +1166,10 @@ private void assertLastCommandSequenceNumber( } @SuppressWarnings({"unchecked", "rawtypes"}) - private static Matcher[] prependWithKey(final List values) { + private static Matcher[] prependWithKey(final String key, final List values) { final Matcher[] allMatchers = new Matcher[values.size() + 1]; - allMatchers[0] = is(values.get(0).toString()); // key + allMatchers[0] = is(key); for (int idx = 0; idx != values.size(); ++idx) { allMatchers[idx + 1] = is(values.get(idx).toString()); diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java index b2bb1d5a43f2..96ef35f72761 100644 --- a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java @@ -125,7 +125,6 @@ public class ConsoleTest { Collections.emptyList(), buildTestSchema(SqlTypes.INTEGER, SqlTypes.STRING), DataSourceType.KTABLE.getKsqlType(), - "key", "2000-01-01", "stats", "errors", @@ -383,7 +382,6 @@ public void testPrintSourceDescription() { writeQueries, fields, DataSourceType.KTABLE.getKsqlType(), - "key", "2000-01-01", "stats", "errors", @@ -516,7 +514,6 @@ public void testPrintSourceDescription() { + " }" + NEWLINE + " } ]," + NEWLINE + " \"type\" : \"TABLE\"," + NEWLINE - + " \"key\" : \"key\"," + NEWLINE + " \"timestamp\" : \"2000-01-01\"," + NEWLINE + " \"statistics\" : \"stats\"," + NEWLINE + " \"errorStats\" : \"errors\"," + NEWLINE @@ -655,7 +652,6 @@ public void testPrintConnectorDescription() { + " }" + NEWLINE + " } ]," + NEWLINE + " \"type\" : \"TABLE\"," + NEWLINE - + " \"key\" : \"key\"," + NEWLINE + " \"timestamp\" : \"2000-01-01\"," + NEWLINE + " \"statistics\" : \"stats\"," + NEWLINE + " \"errorStats\" : \"errors\"," + NEWLINE @@ -1036,7 +1032,6 @@ public void shouldPrintTopicDescribeExtended() { writeQueries, buildTestSchema(SqlTypes.STRING), DataSourceType.KTABLE.getKsqlType(), - "key", "2000-01-01", "stats", "errors", @@ -1104,7 +1099,6 @@ public void shouldPrintTopicDescribeExtended() { + " }" + NEWLINE + " } ]," + NEWLINE + " \"type\" : \"TABLE\"," + NEWLINE - + " \"key\" : \"key\"," + NEWLINE + " \"timestamp\" : \"2000-01-01\"," + NEWLINE + " \"statistics\" : \"stats\"," + NEWLINE + " \"errorStats\" : \"errors\"," + NEWLINE @@ -1122,7 +1116,6 @@ public void shouldPrintTopicDescribeExtended() { assertThat(output, is("" + NEWLINE + "Name : TestSource" + NEWLINE + "Type : TABLE" + NEWLINE - + "Key field : key" + NEWLINE + "Timestamp field : 2000-01-01" + NEWLINE + "Key format : kafka" + NEWLINE + "Value format : avro" + NEWLINE diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java index a63f8409ac5d..3f057c1713f7 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java @@ -29,7 +29,6 @@ */ public final class CreateConfigs { - public static final String KEY_NAME_PROPERTY = "KEY"; public static final String WINDOW_TYPE_PROPERTY = "WINDOW_TYPE"; public static final String WINDOW_SIZE_PROPERTY = "WINDOW_SIZE"; public static final String SCHEMA_ID = "SCHEMA_ID"; @@ -37,13 +36,6 @@ public final class CreateConfigs { private static final ConfigDef CONFIG_DEF = new ConfigDef() .define( - KEY_NAME_PROPERTY, - ConfigDef.Type.STRING, - null, - Importance.MEDIUM, - "The name of a field within the Kafka record value that matches the key. " - + "This may be used by KSQL to avoid unnecessary repartitions." - ).define( WINDOW_TYPE_PROPERTY, ConfigDef.Type.STRING, null, diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/schema/ksql/ColumnMatchers.java b/ksqldb-common/src/test/java/io/confluent/ksql/schema/ksql/ColumnMatchers.java index 97054bf48548..ef75d2e1b021 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/schema/ksql/ColumnMatchers.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/schema/ksql/ColumnMatchers.java @@ -19,10 +19,8 @@ import static org.hamcrest.Matchers.is; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.Column.Namespace; import io.confluent.ksql.schema.ksql.types.SqlType; -import java.util.Optional; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -64,30 +62,6 @@ public static Matcher valueColumn( ); } - public static Matcher valueColumn( - final SourceName source, - final ColumnName name, - final SqlType type - ) { - return allOf( - hasName(name), - hasType(type), - hasNamespace(Namespace.VALUE) - ); - } - - public static Matcher valueColumn( - final Optional source, - final ColumnName name, - final SqlType type - ) { - return allOf( - hasName(name), - hasType(type), - hasNamespace(Namespace.VALUE) - ); - } - public static Matcher hasName(final ColumnName name) { return new FeatureMatcher( is(name), diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java index c20d84d7b2a6..0b0f2a76abda 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java @@ -26,14 +26,12 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.properties.with.CreateSourceProperties; -import io.confluent.ksql.parser.tree.CreateSource; import io.confluent.ksql.parser.tree.CreateStream; import io.confluent.ksql.parser.tree.CreateTable; import io.confluent.ksql.parser.tree.TableElements; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.ksql.SystemColumns; -import io.confluent.ksql.schema.utils.FormatOptions; import io.confluent.ksql.serde.Format; import io.confluent.ksql.serde.GenericKeySerDe; import io.confluent.ksql.serde.GenericRowSerDe; @@ -85,8 +83,7 @@ public CreateStreamCommand createStreamCommand( ) { final SourceName sourceName = statement.getName(); final KsqlTopic topic = buildTopic(statement.getProperties(), serviceContext); - final LogicalSchema schema = buildSchema(statement.getElements(), ksqlConfig); - final Optional keyFieldName = buildKeyFieldName(statement, schema); + final LogicalSchema schema = buildSchema(statement.getElements()); final Optional timestampColumn = buildTimestampColumn( ksqlConfig, statement.getProperties(), @@ -105,7 +102,6 @@ public CreateStreamCommand createStreamCommand( return new CreateStreamCommand( sourceName, schema, - keyFieldName, timestampColumn, topic.getKafkaTopicName(), io.confluent.ksql.execution.plan.Formats @@ -120,8 +116,7 @@ public CreateTableCommand createTableCommand( ) { final SourceName sourceName = statement.getName(); final KsqlTopic topic = buildTopic(statement.getProperties(), serviceContext); - final LogicalSchema schema = buildSchema(statement.getElements(), ksqlConfig); - final Optional keyFieldName = buildKeyFieldName(statement, schema); + final LogicalSchema schema = buildSchema(statement.getElements()); final Optional timestampColumn = buildTimestampColumn( ksqlConfig, statement.getProperties(), @@ -139,7 +134,6 @@ public CreateTableCommand createTableCommand( return new CreateTableCommand( sourceName, schema, - keyFieldName, timestampColumn, topic.getKafkaTopicName(), io.confluent.ksql.execution.plan.Formats @@ -148,26 +142,7 @@ public CreateTableCommand createTableCommand( ); } - private static Optional buildKeyFieldName( - final CreateSource statement, - final LogicalSchema schema) { - if (statement.getProperties().getKeyField().isPresent()) { - final ColumnName column = statement.getProperties().getKeyField().get(); - schema.findValueColumn(column) - .orElseThrow(() -> new KsqlException( - "The KEY column set in the WITH clause does not exist in the schema: '" - + column.toString(FormatOptions.noEscape()) + "'" - )); - return Optional.of(column); - } else { - return Optional.empty(); - } - } - - private static LogicalSchema buildSchema( - final TableElements tableElements, - final KsqlConfig ksqlConfig - ) { + private static LogicalSchema buildSchema(final TableElements tableElements) { if (Iterables.isEmpty(tableElements)) { throw new KsqlException("The statement does not define any columns."); } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandExec.java b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandExec.java index 36ee593b770f..afb9a1004130 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandExec.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandExec.java @@ -26,16 +26,13 @@ import io.confluent.ksql.execution.ddl.commands.RegisterTypeCommand; import io.confluent.ksql.metastore.MutableMetaStore; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.metastore.model.KsqlTable; -import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.serde.ValueFormat; import java.util.Objects; -import java.util.Optional; /** * Execute DDL Commands @@ -74,7 +71,6 @@ public DdlCommandResult executeCreateStream(final CreateStreamCommand createStre createStream.getSourceName(), createStream.getSchema(), createStream.getFormats().getOptions(), - getKeyField(createStream.getKeyField()), createStream.getTimestampColumn(), withQuery, getKsqlTopic(createStream) @@ -90,7 +86,6 @@ public DdlCommandResult executeCreateTable(final CreateTableCommand createTable) createTable.getSourceName(), createTable.getSchema(), createTable.getFormats().getOptions(), - getKeyField(createTable.getKeyField()), createTable.getTimestampColumn(), withQuery, getKsqlTopic(createTable) @@ -132,12 +127,6 @@ public DdlCommandResult executeDropType(final DropTypeCommand dropType) { } } - private static KeyField getKeyField(final Optional keyFieldName) { - return keyFieldName - .map(columnName -> KeyField.of(columnName)) - .orElseGet(KeyField::none); - } - private static KsqlTopic getKsqlTopic(final CreateSourceCommand createSource) { return new KsqlTopic( createSource.getTopicName(), diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java index 2efac26af8bb..24dc6743db5b 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java @@ -26,7 +26,6 @@ import io.confluent.ksql.execution.plan.ExecutionStep; import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.CreateStreamAsSelect; import io.confluent.ksql.parser.tree.CreateTableAsSelect; @@ -42,7 +41,6 @@ import io.confluent.ksql.planner.plan.PlanNode; import io.confluent.ksql.query.QueryExecutor; import io.confluent.ksql.query.QueryId; -import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.statement.ConfiguredStatement; @@ -165,8 +163,8 @@ KsqlPlan plan(final ConfiguredStatement statement) { final Optional ddlCommand = maybeCreateSinkDdl( statement.getStatementText(), - outputNode, - plans.physicalPlan.getKeyField().get()); + outputNode + ); validateQuery(outputNode.getNodeOutputType(), statement); @@ -227,11 +225,10 @@ private ExecutorPlans( private Optional maybeCreateSinkDdl( final String sql, - final KsqlStructuredDataOutputNode outputNode, - final KeyField keyField + final KsqlStructuredDataOutputNode outputNode ) { if (!outputNode.isDoCreateInto()) { - validateExistingSink(outputNode, keyField); + validateExistingSink(outputNode); return Optional.empty(); } @@ -246,7 +243,6 @@ private Optional maybeCreateSinkDdl( ddl = new CreateStreamCommand( outputNode.getIntoSourceName(), outputNode.getSchema(), - keyField.ref(), outputNode.getTimestampColumn(), outputNode.getKsqlTopic().getKafkaTopicName(), formats, @@ -256,7 +252,6 @@ private Optional maybeCreateSinkDdl( ddl = new CreateTableCommand( outputNode.getIntoSourceName(), outputNode.getSchema(), - keyField.ref(), outputNode.getTimestampColumn(), outputNode.getKsqlTopic().getKafkaTopicName(), formats, @@ -270,8 +265,7 @@ private Optional maybeCreateSinkDdl( } private void validateExistingSink( - final KsqlStructuredDataOutputNode outputNode, - final KeyField keyField + final KsqlStructuredDataOutputNode outputNode ) { final SourceName name = outputNode.getIntoSourceName(); final DataSource existing = engineContext.getMetaStore().getSource(name); @@ -300,43 +294,6 @@ private void validateExistingSink( + "Sink schema is " + existingSchema ); } - - enforceKeyEquivalence( - existing.getKeyField().resolve(existingSchema), - keyField.resolve(resultSchema) - ); - } - - private static void enforceKeyEquivalence( - final Optional sinkKeyCol, - final Optional resultKeyCol - ) { - if (!sinkKeyCol.isPresent() && !resultKeyCol.isPresent()) { - return; - } - - if (sinkKeyCol.isPresent() - && resultKeyCol.isPresent() - && sinkKeyCol.get().name().equals(resultKeyCol.get().name()) - && Objects.equals(sinkKeyCol.get().type(), resultKeyCol.get().type())) { - return; - } - - throwIncompatibleKeysException(sinkKeyCol, resultKeyCol); - } - - private static void throwIncompatibleKeysException( - final Optional sinkKeyCol, - final Optional resultKeyCol - ) { - throw new KsqlException(String.format( - "Incompatible key fields for sink and results. Sink" - + " key field is %s (type: %s) while result key " - + "field is %s (type: %s)", - sinkKeyCol.map(c -> c.name().text()).orElse(null), - sinkKeyCol.map(Column::type).orElse(null), - resultKeyCol.map(c -> c.name().text()).orElse(null), - resultKeyCol.map(Column::type).orElse(null))); } private static void validateQuery( diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index de633fd2131f..743de5322722 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -34,7 +34,6 @@ import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.tree.InsertValues; import io.confluent.ksql.rest.SessionProperties; @@ -275,12 +274,6 @@ private RowData extractRow( final Map values = resolveValues( insertValues, columns, schemaWithRowTime, functionRegistry, config); - handleExplicitKeyField( - values, - dataSource.getKeyField(), - Iterables.getOnlyElement(schemaWithRowTime.key()) - ); - if (dataSource.getDataSourceType() == DataSourceType.KTABLE) { final String noValue = schemaWithRowTime.key().stream() .map(Column::name) @@ -384,33 +377,6 @@ private static Map resolveValues( return values; } - private static void handleExplicitKeyField( - final Map values, - final KeyField keyField, - final Column keyColumn - ) { - // key column: the key column in the source's schema. - // key field: the column identified in the WITH clause as being an alias to the key column. - - keyField.ref().ifPresent(keyFieldName -> { - final ColumnName keyColumnName = keyColumn.name(); - final Object keyFieldValue = values.get(keyFieldName); - final Object keyColumnValue = values.get(keyColumnName); - - if (keyFieldValue != null ^ keyColumnValue != null) { - if (keyFieldValue == null) { - values.put(keyFieldName, keyColumnValue); - } else { - values.put(keyColumnName, keyFieldValue); - } - } else if (keyFieldValue != null && !Objects.equals(keyFieldValue, keyColumnValue)) { - throw new KsqlException( - "Expected " + keyColumnName.text() + " and " + keyFieldName.text() + " to match " - + "but got " + keyColumnValue + " and " + keyFieldValue + " respectively."); - } - }); - } - private static SqlType columnType(final ColumnName column, final LogicalSchema schema) { return schema .findColumn(column) diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlan.java b/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlan.java index aba68028c925..6cbd3b76936e 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlan.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlan.java @@ -15,39 +15,28 @@ package io.confluent.ksql.physical; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.plan.ExecutionStep; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.query.QueryId; import java.util.Objects; -import java.util.Optional; @Immutable public final class PhysicalPlan { private final QueryId queryId; private final ExecutionStep physicalPlan; - private final Optional keyField; PhysicalPlan( final QueryId queryId, - final ExecutionStep physicalPlan, - final Optional keyField + final ExecutionStep physicalPlan ) { this.queryId = Objects.requireNonNull(queryId, "queryId"); this.physicalPlan = Objects.requireNonNull(physicalPlan, "physicalPlan"); - this.keyField = Objects.requireNonNull(keyField, "keyField"); } public ExecutionStep getPhysicalPlan() { return physicalPlan; } - @JsonIgnore - public Optional getKeyField() { - return keyField; - } - public QueryId getQueryId() { return queryId; } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java b/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java index 997096ff6d0e..7839bc21fad6 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java @@ -26,7 +26,6 @@ import io.confluent.ksql.structured.SchemaKStream; import io.confluent.ksql.util.KsqlConfig; import java.util.Objects; -import java.util.Optional; import org.apache.kafka.streams.StreamsBuilder; public class PhysicalPlanBuilder { @@ -73,8 +72,7 @@ public PhysicalPlan buildPhysicalPlan(final LogicalPlanNode logicalPlanNode) { final SchemaKStream resultStream = outputNode.buildStream(ksqlQueryBuilder); return new PhysicalPlan( queryId, - resultStream.getSourceStep(), - Optional.of(resultStream.getKeyField()) + resultStream.getSourceStep() ); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java index 6fc60837b5a2..f8a14b35452a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java @@ -41,7 +41,6 @@ import io.confluent.ksql.execution.util.ExpressionTypeManager; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.udf.AsValue; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.AllColumns; @@ -70,7 +69,6 @@ import io.confluent.ksql.schema.ksql.ColumnNames; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.LogicalSchema.Builder; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.Format; @@ -160,7 +158,6 @@ private OutputNode buildOutputNode(final PlanNode sourcePlanNode) { sourcePlanNode, inputSchema, timestampColumn, - sourcePlanNode.getKeyField(), intoDataSource.getKsqlTopic(), analysis.getLimitClause(), intoDataSource.isCreate(), @@ -210,9 +207,6 @@ private AggregateNode buildAggregateNode(final PlanNode sourcePlanNode) { final GroupBy groupBy = analysis.getGroupBy() .orElseThrow(IllegalStateException::new); - final List groupByExps = groupBy - .getGroupingExpressions(); - final List projectionExpressions = buildSelectExpressions( sourcePlanNode, analysis.getSelectItems() @@ -221,17 +215,6 @@ private AggregateNode buildAggregateNode(final PlanNode sourcePlanNode) { final LogicalSchema schema = buildAggregateSchema(sourcePlanNode, groupBy, projectionExpressions); - final Expression groupBySingle = groupByExps.size() == 1 - ? groupByExps.get(0) - : null; - - final Optional keyFieldName = getSelectAliasMatching( - (expression, alias) -> - expression.equals(groupBySingle) - && !SystemColumns.isSystemColumn(alias) - && !schema.isKeyColumn(alias), - projectionExpressions); - final RewrittenAggregateAnalysis aggregateAnalysis = new RewrittenAggregateAnalysis( aggregateAnalyzer.analyze(analysis, projectionExpressions), refRewriter::process @@ -250,7 +233,6 @@ private AggregateNode buildAggregateNode(final PlanNode sourcePlanNode) { new PlanNodeId("Aggregate"), sourcePlanNode, schema, - keyFieldName, groupBy, functionRegistry, analysis, @@ -429,33 +411,6 @@ private RepartitionNode buildRepartitionNode( final Expression rewrittenPartitionBy = ExpressionTreeRewriter.rewriteWith(plugin, partitionBy); - final KeyField keyField; - - if (!(rewrittenPartitionBy instanceof UnqualifiedColumnReferenceExp)) { - keyField = KeyField.none(); - } else { - final ColumnName columnName = ((UnqualifiedColumnReferenceExp) rewrittenPartitionBy) - .getColumnName(); - - final LogicalSchema sourceSchema = sourceNode.getSchema(); - - final Column proposedKey = sourceSchema - .findColumn(columnName) - .orElseThrow(IllegalStateException::new); - - switch (proposedKey.namespace()) { - case KEY: - keyField = sourceNode.getKeyField(); - break; - case VALUE: - keyField = KeyField.of(columnName); - break; - default: - keyField = KeyField.none(); - break; - } - } - final LogicalSchema schema = buildRepartitionedSchema(sourceNode, rewrittenPartitionBy); @@ -465,7 +420,6 @@ private RepartitionNode buildRepartitionNode( schema, partitionBy, rewrittenPartitionBy, - keyField, internal ); } @@ -640,19 +594,6 @@ private static DataSourceNode buildNonJoinNode(final AliasedDataSource dataSourc ); } - private static Optional getSelectAliasMatching( - final BiFunction matcher, - final List projection - ) { - for (final SelectExpression select : projection) { - if (matcher.apply(select.getExpression(), select.getAlias())) { - return Optional.of(select.getAlias()); - } - } - - return Optional.empty(); - } - private LogicalSchema buildProjectionSchema( final LogicalSchema parentSchema, final List projection @@ -784,30 +725,6 @@ private LogicalSchema buildRepartitionedSchema( ); } - private static boolean exactlyMatchesKeyColumns( - final Expression expression, - final LogicalSchema schema - ) { - if (schema.key().size() != 1) { - // Currently only support single key column: - return false; - } - - if (!(expression instanceof ColumnReferenceExp)) { - // Anything not a column ref can't be a match: - return false; - } - - final ColumnName columnName = ((ColumnReferenceExp) expression).getColumnName(); - - final Namespace ns = schema - .findColumn(columnName) - .map(Column::namespace) - .orElse(Namespace.VALUE); - - return ns == Namespace.KEY; - } - private static List selectWithPrependAlias( final SourceName alias, final LogicalSchema schema diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java index ae160c995a9e..4902e171082f 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java @@ -37,7 +37,6 @@ import io.confluent.ksql.execution.expression.tree.VisitParentExpressionVisitor; import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.function.FunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.GroupBy; @@ -73,7 +72,6 @@ public class AggregateNode extends PlanNode implements VerifiableNode { private static final String PROJECT_OP_NAME = "Project"; private final PlanNode source; - private final KeyField keyField; private final GroupBy groupBy; private final Optional windowExpression; private final ImmutableList aggregateFunctionArguments; @@ -88,7 +86,6 @@ public AggregateNode( final PlanNodeId id, final PlanNode source, final LogicalSchema schema, - final Optional keyFieldName, final GroupBy groupBy, final FunctionRegistry functionRegistry, final ImmutableAnalysis analysis, @@ -126,19 +123,12 @@ public AggregateNode( this.havingExpressions = rewrittenAggregateAnalysis.getHavingExpression() .map(exp -> ExpressionTreeRewriter.rewriteWith(aggregateExpressionRewriter::process, exp)); - this.keyField = KeyField.of(requireNonNull(keyFieldName, "keyFieldName")) - .validateKeyExistsIn(schema); this.valueFormat = getTheSourceNode() .getDataSource() .getKsqlTopic() .getValueFormat(); } - @Override - public KeyField getKeyField() { - return keyField; - } - @Override public List getSources() { return ImmutableList.of(source); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java index 333b93871016..a51d2cfb9e0e 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java @@ -29,7 +29,6 @@ import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.Projection; @@ -47,7 +46,6 @@ public class DataSourceNode extends PlanNode { private static final String SOURCE_OP_NAME = "Source"; private final DataSource dataSource; - private final KeyField keyField; private final SchemaKStreamFactory schemaKStreamFactory; public DataSourceNode( @@ -67,17 +65,9 @@ public DataSourceNode( super(id, dataSource.getDataSourceType(), buildSchema(dataSource), Optional.of(alias)); this.dataSource = requireNonNull(dataSource, "dataSource"); - this.keyField = dataSource.getKeyField() - .validateKeyExistsIn(getSchema()); - this.schemaKStreamFactory = requireNonNull(schemaKStreamFactory, "schemaKStreamFactory"); } - @Override - public KeyField getKeyField() { - return keyField; - } - public DataSource getDataSource() { return dataSource; } @@ -115,8 +105,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { return schemaKStreamFactory.create( builder, dataSource, - contextStacker.push(SOURCE_OP_NAME), - keyField + contextStacker.push(SOURCE_OP_NAME) ); } @@ -159,8 +148,7 @@ interface SchemaKStreamFactory { SchemaKStream create( KsqlQueryBuilder builder, DataSource dataSource, - QueryContext.Stacker contextStacker, - KeyField keyField + QueryContext.Stacker contextStacker ); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java index a8ff1a96484d..55ebbe07097c 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java @@ -20,7 +20,6 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext.Stacker; import io.confluent.ksql.execution.expression.tree.Expression; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.services.KafkaTopicClient; import io.confluent.ksql.structured.SchemaKStream; import java.util.List; @@ -47,11 +46,6 @@ public Expression getPredicate() { return predicate; } - @Override - public KeyField getKeyField() { - return source.getKeyField(); - } - @Override public List getSources() { return ImmutableList.of(source); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java index f534f7678db6..cc0ef21f3ded 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java @@ -29,7 +29,6 @@ import io.confluent.ksql.execution.expression.tree.VisitParentExpressionVisitor; import io.confluent.ksql.execution.streams.StreamFlatMapBuilder; import io.confluent.ksql.function.FunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.parser.tree.SelectItem; @@ -71,11 +70,6 @@ public FlatMapNode( this.columnMappings = buildColumnMappings(functionRegistry, analysis); } - @Override - public KeyField getKeyField() { - return source.getKeyField(); - } - @Override public List getSources() { return ImmutableList.of(source); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java index bc649048d7d0..28df5c37916c 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java @@ -35,7 +35,6 @@ import io.confluent.ksql.execution.streams.JoinParamsFactory; import io.confluent.ksql.function.udf.JoinKeyUdf; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.WithinExpression; @@ -72,7 +71,6 @@ public enum JoinType { private final boolean finalJoin; private final PlanNode left; private final PlanNode right; - private final KeyField keyField; private final Optional withinExpression; public JoinNode( @@ -97,14 +95,6 @@ public JoinNode( this.left = requireNonNull(left, "left"); this.right = requireNonNull(right, "right"); this.withinExpression = requireNonNull(withinExpression, "withinExpression"); - this.keyField = joinType == JoinType.OUTER - ? KeyField.none() // Both source key columns can be null, hence neither can be the keyField - : left.getKeyField(); - } - - @Override - public KeyField getKeyField() { - return keyField; } @Override @@ -323,7 +313,6 @@ public SchemaKStream join() { return leftStream.leftJoin( rightStream, joinNode.getKeyColumnName(), - joinNode.keyField, joinNode.withinExpression.get().joinWindow(), getFormatForSource(joinNode.left), getFormatForSource(joinNode.right), @@ -333,7 +322,6 @@ public SchemaKStream join() { return leftStream.outerJoin( rightStream, joinNode.getKeyColumnName(), - joinNode.keyField, joinNode.withinExpression.get().joinWindow(), getFormatForSource(joinNode.left), getFormatForSource(joinNode.right), @@ -343,7 +331,6 @@ public SchemaKStream join() { return leftStream.join( rightStream, joinNode.getKeyColumnName(), - joinNode.keyField, joinNode.withinExpression.get().joinWindow(), getFormatForSource(joinNode.left), getFormatForSource(joinNode.right), @@ -383,7 +370,6 @@ public SchemaKStream join() { return leftStream.leftJoin( rightTable, joinNode.getKeyColumnName(), - joinNode.keyField, getFormatForSource(joinNode.left), contextStacker ); @@ -392,7 +378,6 @@ public SchemaKStream join() { return leftStream.join( rightTable, joinNode.getKeyColumnName(), - joinNode.keyField, getFormatForSource(joinNode.left), contextStacker ); @@ -432,19 +417,16 @@ public SchemaKTable join() { return leftTable.leftJoin( rightTable, joinNode.getKeyColumnName(), - joinNode.keyField, contextStacker); case INNER: return leftTable.join( rightTable, joinNode.getKeyColumnName(), - joinNode.keyField, contextStacker); case OUTER: return leftTable.outerJoin( rightTable, joinNode.getKeyColumnName(), - joinNode.keyField, contextStacker); default: throw new KsqlException("Invalid join type encountered: " + joinNode.joinType); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlBareOutputNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlBareOutputNode.java index 441273a326c3..79f85b79b7ca 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlBareOutputNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlBareOutputNode.java @@ -17,7 +17,6 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.timestamp.TimestampColumn; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.query.id.QueryIdGenerator; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -29,8 +28,6 @@ @SuppressWarnings("OptionalUsedAsFieldOrParameterType") public class KsqlBareOutputNode extends OutputNode { - private final KeyField keyField; - public KsqlBareOutputNode( final PlanNodeId id, final PlanNode source, @@ -39,8 +36,6 @@ public KsqlBareOutputNode( final Optional timestampColumn ) { super(id, source, schema, limit, timestampColumn); - this.keyField = KeyField.of(source.getKeyField().ref()) - .validateKeyExistsIn(schema); } @Override @@ -48,11 +43,6 @@ public QueryId getQueryId(final QueryIdGenerator queryIdGenerator) { return new QueryId(String.valueOf(Math.abs(ThreadLocalRandom.current().nextLong()))); } - @Override - public KeyField getKeyField() { - return keyField; - } - @Override public SchemaKStream buildStream(final KsqlQueryBuilder builder) { return getSource().buildStream(builder); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java index abec0b910c28..61e155b885b2 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java @@ -23,7 +23,6 @@ import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.execution.timestamp.TimestampColumn; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.query.QueryId; @@ -43,7 +42,6 @@ public class KsqlStructuredDataOutputNode extends OutputNode { private final KsqlTopic ksqlTopic; - private final KeyField keyField; private final boolean doCreateInto; private final ImmutableSet serdeOptions; private final SourceName intoSourceName; @@ -54,7 +52,6 @@ public KsqlStructuredDataOutputNode( final PlanNode source, final LogicalSchema schema, final Optional timestampColumn, - final KeyField keyField, final KsqlTopic ksqlTopic, final OptionalInt limit, final boolean doCreateInto, @@ -74,8 +71,6 @@ public KsqlStructuredDataOutputNode( ); this.serdeOptions = ImmutableSet.copyOf(requireNonNull(serdeOptions, "serdeOptions")); - this.keyField = requireNonNull(keyField, "keyField") - .validateKeyExistsIn(schema); this.ksqlTopic = requireNonNull(ksqlTopic, "ksqlTopic"); this.doCreateInto = doCreateInto; this.intoSourceName = requireNonNull(intoSourceName, "intoSourceName"); @@ -111,11 +106,6 @@ public QueryId getQueryId(final QueryIdGenerator queryIdGenerator) { return new QueryId("CSAS_" + getId().toString().toUpperCase() + "_" + base); } - @Override - public KeyField getKeyField() { - return keyField; - } - @Override public SchemaKStream buildStream(final KsqlQueryBuilder builder) { final PlanNode source = getSource(); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java index 2a4d1ebc5bb9..49b99da88cd5 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java @@ -22,7 +22,6 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.Projection; @@ -69,8 +68,6 @@ public final LogicalSchema getSchema() { return schema; } - public abstract KeyField getKeyField(); - public abstract List getSources(); public R accept(final PlanVisitor visitor, final C context) { diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java index 7dc9cfd5da41..83de478c6cc7 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java @@ -25,7 +25,6 @@ import io.confluent.ksql.execution.expression.tree.ColumnReferenceExp; import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.function.udf.AsValue; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.Column; @@ -43,7 +42,6 @@ public class ProjectNode extends PlanNode { private final PlanNode source; private final ImmutableList selectExpressions; - private final KeyField keyField; private final ImmutableMap aliases; public ProjectNode( @@ -60,8 +58,6 @@ public ProjectNode( this.selectExpressions = ImmutableList .copyOf(requireNonNull(projectExpressions, "projectExpressions")); - this.keyField = KeyField.none(); - this.aliases = aliased ? buildAliasMapping(projectExpressions) : ImmutableMap.of(); @@ -83,11 +79,6 @@ protected int getPartitions(final KafkaTopicClient kafkaTopicClient) { return source.getPartitions(kafkaTopicClient); } - @Override - public KeyField getKeyField() { - return keyField; - } - public List getSelectExpressions() { return selectExpressions; } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java index 039f9e0738b1..948dc554edb8 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java @@ -25,7 +25,6 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.Projection; @@ -42,7 +41,6 @@ public class RepartitionNode extends PlanNode { private final PlanNode source; private final Expression originalPartitionBy; private final Expression partitionBy; - private final KeyField keyField; private final boolean internal; public RepartitionNode( @@ -51,22 +49,15 @@ public RepartitionNode( final LogicalSchema schema, final Expression originalPartitionBy, final Expression partitionBy, - final KeyField keyField, final boolean internal ) { super(id, source.getNodeOutputType(), schema, source.getSourceName()); this.source = requireNonNull(source, "source"); this.originalPartitionBy = requireNonNull(originalPartitionBy, "originalPartitionBy"); this.partitionBy = requireNonNull(partitionBy, "partitionBy"); - this.keyField = requireNonNull(keyField, "keyField"); this.internal = internal; } - @Override - public KeyField getKeyField() { - return keyField; - } - @Override public List getSources() { return ImmutableList.of(source); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedStream.java b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedStream.java index 5bbd788e8d2a..1bd196265fef 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedStream.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedStream.java @@ -23,7 +23,6 @@ import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.execution.streams.StepSchemaResolver; import io.confluent.ksql.function.FunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.tree.WindowExpression; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -42,7 +41,6 @@ public class SchemaKGroupedStream { final ExecutionStep sourceStep; final LogicalSchema schema; final KeyFormat keyFormat; - final KeyField keyField; final KsqlConfig ksqlConfig; final FunctionRegistry functionRegistry; @@ -50,22 +48,16 @@ public class SchemaKGroupedStream { final ExecutionStep sourceStep, final LogicalSchema schema, final KeyFormat keyFormat, - final KeyField keyField, final KsqlConfig ksqlConfig, final FunctionRegistry functionRegistry ) { this.sourceStep = sourceStep; this.schema = Objects.requireNonNull(schema, "schema"); this.keyFormat = Objects.requireNonNull(keyFormat, "keyFormat"); - this.keyField = keyField; this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); this.functionRegistry = functionRegistry; } - public KeyField getKeyField() { - return keyField; - } - public ExecutionStep getSourceStep() { return sourceStep; } @@ -106,7 +98,6 @@ public SchemaKTable aggregate( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedTable.java b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedTable.java index d2ef19d979ef..0f47aa9a3ead 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedTable.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKGroupedTable.java @@ -26,7 +26,6 @@ import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.KsqlAggregateFunction; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.parser.tree.WindowExpression; @@ -44,13 +43,13 @@ import org.apache.kafka.connect.data.Struct; public class SchemaKGroupedTable extends SchemaKGroupedStream { + private final ExecutionStep sourceTableStep; SchemaKGroupedTable( final ExecutionStep sourceTableStep, final LogicalSchema schema, final KeyFormat keyFormat, - final KeyField keyField, final KsqlConfig ksqlConfig, final FunctionRegistry functionRegistry ) { @@ -58,7 +57,6 @@ public class SchemaKGroupedTable extends SchemaKGroupedStream { null, schema, keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -110,7 +108,6 @@ public SchemaKTable aggregate( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKSourceFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKSourceFactory.java index 549ef90506ec..7f27e01ec01a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKSourceFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKSourceFactory.java @@ -30,7 +30,6 @@ import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.execution.streams.StepSchemaResolver; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.serde.WindowInfo; @@ -46,8 +45,7 @@ private SchemaKSourceFactory() { public static SchemaKStream buildSource( final KsqlQueryBuilder builder, final DataSource dataSource, - final QueryContext.Stacker contextStacker, - final KeyField keyField + final QueryContext.Stacker contextStacker ) { final boolean windowed = dataSource.getKsqlTopic().getKeyFormat().isWindowed(); switch (dataSource.getDataSourceType()) { @@ -56,13 +54,11 @@ public static SchemaKStream buildSource( ? buildWindowedStream( builder, dataSource, - contextStacker, - keyField + contextStacker ) : buildStream( builder, dataSource, - contextStacker, - keyField + contextStacker ); case KTABLE: @@ -70,13 +66,11 @@ public static SchemaKStream buildSource( ? buildWindowedTable( builder, dataSource, - contextStacker, - keyField + contextStacker ) : buildTable( builder, dataSource, - contextStacker, - keyField + contextStacker ); default: @@ -87,8 +81,7 @@ public static SchemaKStream buildSource( private static SchemaKStream buildWindowedStream( final KsqlQueryBuilder builder, final DataSource dataSource, - final Stacker contextStacker, - final KeyField keyField + final Stacker contextStacker ) { final WindowInfo windowInfo = dataSource.getKsqlTopic().getKeyFormat().getWindowInfo() .orElseThrow(IllegalArgumentException::new); @@ -106,16 +99,14 @@ private static SchemaKStream buildWindowedStream( builder, resolveSchema(builder, step, dataSource), dataSource.getKsqlTopic().getKeyFormat(), - step, - keyField + step ); } private static SchemaKStream buildStream( final KsqlQueryBuilder builder, final DataSource dataSource, - final Stacker contextStacker, - final KeyField keyField + final Stacker contextStacker ) { if (dataSource.getKsqlTopic().getKeyFormat().getWindowInfo().isPresent()) { throw new IllegalArgumentException("windowed"); @@ -133,16 +124,14 @@ private static SchemaKStream buildStream( builder, resolveSchema(builder, step, dataSource), dataSource.getKsqlTopic().getKeyFormat(), - step, - keyField + step ); } private static SchemaKTable buildWindowedTable( final KsqlQueryBuilder builder, final DataSource dataSource, - final Stacker contextStacker, - final KeyField keyField + final Stacker contextStacker ) { final WindowInfo windowInfo = dataSource.getKsqlTopic().getKeyFormat().getWindowInfo() .orElseThrow(IllegalArgumentException::new); @@ -160,16 +149,14 @@ private static SchemaKTable buildWindowedTable( builder, resolveSchema(builder, step, dataSource), dataSource.getKsqlTopic().getKeyFormat(), - step, - keyField + step ); } private static SchemaKTable buildTable( final KsqlQueryBuilder builder, final DataSource dataSource, - final Stacker contextStacker, - final KeyField keyField + final Stacker contextStacker ) { if (dataSource.getKsqlTopic().getKeyFormat().getWindowInfo().isPresent()) { throw new IllegalArgumentException("windowed"); @@ -187,8 +174,7 @@ private static SchemaKTable buildTable( builder, resolveSchema(builder, step, dataSource), dataSource.getKsqlTopic().getKeyFormat(), - step, - keyField + step ); } @@ -196,14 +182,12 @@ private static SchemaKStream schemaKStream( final KsqlQueryBuilder builder, final LogicalSchema schema, final KeyFormat keyFormat, - final SourceStep> streamSource, - final KeyField keyField + final SourceStep> streamSource ) { return new SchemaKStream<>( streamSource, schema, keyFormat, - keyField, builder.getKsqlConfig(), builder.getFunctionRegistry() ); @@ -213,14 +197,12 @@ private static SchemaKTable schemaKTable( final KsqlQueryBuilder builder, final LogicalSchema schema, final KeyFormat keyFormat, - final SourceStep> tableSource, - final KeyField keyField + final SourceStep> tableSource ) { return new SchemaKTable<>( tableSource, schema, keyFormat, - keyField, builder.getKsqlConfig(), builder.getFunctionRegistry() ); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java index 514b5f2703da..e3682c4f3b47 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java @@ -25,7 +25,6 @@ import io.confluent.ksql.execution.expression.tree.ColumnReferenceExp; import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.execution.expression.tree.FunctionCall; -import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.execution.plan.ExecutionStep; import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.plan.JoinType; @@ -43,12 +42,8 @@ import io.confluent.ksql.execution.streams.StepSchemaResolver; import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.function.FunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; -import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.ValueFormat; @@ -58,7 +53,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.kstream.JoinWindows; @@ -72,7 +66,6 @@ public class SchemaKStream { public enum Type { SOURCE, PROJECT, FILTER, AGGREGATE, SINK, REKEY, JOIN } final KeyFormat keyFormat; - final KeyField keyField; final KsqlConfig ksqlConfig; final FunctionRegistry functionRegistry; final LogicalSchema schema; @@ -82,14 +75,12 @@ public enum Type { SOURCE, PROJECT, FILTER, AGGREGATE, SINK, REKEY, JOIN } final ExecutionStep> sourceStep, final LogicalSchema schema, final KeyFormat keyFormat, - final KeyField keyField, final KsqlConfig ksqlConfig, final FunctionRegistry functionRegistry ) { this.keyFormat = requireNonNull(keyFormat, "keyFormat"); this.sourceStep = sourceStep; this.schema = Objects.requireNonNull(schema, "schema"); - this.keyField = requireNonNull(keyField, "keyField").validateKeyExistsIn(schema); this.ksqlConfig = requireNonNull(ksqlConfig, "ksqlConfig"); this.functionRegistry = requireNonNull(functionRegistry, "functionRegistry"); } @@ -112,7 +103,6 @@ public SchemaKStream into( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -132,7 +122,6 @@ public SchemaKStream filter( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -147,7 +136,6 @@ public SchemaKStream select( final QueryContext.Stacker contextStacker, final KsqlQueryBuilder ksqlQueryBuilder ) { - final KeyField keyField = findKeyField(selectExpressions); final StreamSelect step = ExecutionStepFactory.streamSelect( contextStacker, sourceStep, @@ -158,49 +146,14 @@ public SchemaKStream select( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); } - @SuppressWarnings("deprecation") - KeyField findKeyField(final List selectExpressions) { - if (!getKeyField().ref().isPresent()) { - return KeyField.none(); - } - - final ColumnName keyColumnName = getKeyField().ref().get(); - - Optional found = Optional.empty(); - - for (final SelectExpression selectExpression : selectExpressions) { - final ColumnName toName = selectExpression.getAlias(); - final Expression toExpression = selectExpression.getExpression(); - - if (toExpression instanceof UnqualifiedColumnReferenceExp) { - final UnqualifiedColumnReferenceExp nameRef = (UnqualifiedColumnReferenceExp) toExpression; - - if (keyColumnName.equals(nameRef.getColumnName())) { - found = Optional.of(Column.legacyKeyFieldColumn(toName, SqlTypes.STRING)); - break; - } - } - } - - final Optional filtered = found - // System columns can not be key fields: - .filter(f -> !SystemColumns.isSystemColumn(f.name())) - .map(Column::name); - - return KeyField.of(filtered); - } - - public SchemaKStream leftJoin( final SchemaKTable schemaKTable, final ColumnName keyColName, - final KeyField keyField, final ValueFormat valueFormat, final Stacker contextStacker ) { @@ -217,7 +170,6 @@ public SchemaKStream leftJoin( step, resolveSchema(step, schemaKTable), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -226,7 +178,6 @@ public SchemaKStream leftJoin( public SchemaKStream leftJoin( final SchemaKStream otherSchemaKStream, final ColumnName keyColName, - final KeyField keyField, final JoinWindows joinWindows, final ValueFormat leftFormat, final ValueFormat rightFormat, @@ -247,7 +198,6 @@ public SchemaKStream leftJoin( step, resolveSchema(step, otherSchemaKStream), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -256,7 +206,6 @@ public SchemaKStream leftJoin( public SchemaKStream join( final SchemaKTable schemaKTable, final ColumnName keyColName, - final KeyField keyField, final ValueFormat valueFormat, final Stacker contextStacker ) { @@ -273,7 +222,6 @@ public SchemaKStream join( step, resolveSchema(step, schemaKTable), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -282,7 +230,6 @@ public SchemaKStream join( public SchemaKStream join( final SchemaKStream otherSchemaKStream, final ColumnName keyColName, - final KeyField keyField, final JoinWindows joinWindows, final ValueFormat leftFormat, final ValueFormat rightFormat, @@ -303,7 +250,6 @@ public SchemaKStream join( step, resolveSchema(step, otherSchemaKStream), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -312,7 +258,6 @@ public SchemaKStream join( public SchemaKStream outerJoin( final SchemaKStream otherSchemaKStream, final ColumnName keyColName, - final KeyField keyField, final JoinWindows joinWindows, final ValueFormat leftFormat, final ValueFormat rightFormat, @@ -333,7 +278,6 @@ public SchemaKStream outerJoin( step, resolveSchema(step, otherSchemaKStream), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -360,22 +304,11 @@ public SchemaKStream selectKey( step, resolveSchema(step), keyFormat, - getNewKeyField(keyExpression), ksqlConfig, functionRegistry ); } - private KeyField getNewKeyField(final Expression expression) { - if (!(expression instanceof UnqualifiedColumnReferenceExp)) { - return KeyField.none(); - } - - final ColumnName columnName = ((UnqualifiedColumnReferenceExp) expression).getColumnName(); - final KeyField newKeyField = isKeyColumn(columnName) ? keyField : KeyField.of(columnName); - return SystemColumns.isPseudoColumn(columnName) ? KeyField.none() : newKeyField; - } - boolean repartitionNotNeeded(final List expressions) { // Note: A repartition is only not required if partitioning by the existing key column, or // the existing keyField. @@ -402,11 +335,7 @@ boolean repartitionNotNeeded(final List expressions) { .findValueColumn(newKeyColName) .orElseThrow(IllegalStateException::new); - final boolean matchesKeyField = keyField.resolve(getSchema()) - .map(kf -> kf.name().equals(newKeyColName)) - .orElse(false); - - return matchesKeyField || isKeyColumn(newKeyColName); + return isKeyColumn(newKeyColName); } private boolean isKeyColumn(final ColumnName fieldName) { @@ -427,12 +356,6 @@ public SchemaKGroupedStream groupBy( return groupByKey(rekeyedKeyFormat, valueFormat, contextStacker); } - final ColumnName aggregateKeyName = groupedKeyNameFor(groupByExpressions); - - final Optional newKeyCol = getSchema() - .findValueColumn(aggregateKeyName) - .map(Column::name); - final StreamGroupBy source = ExecutionStepFactory.streamGroupBy( contextStacker, sourceStep, @@ -444,7 +367,6 @@ public SchemaKGroupedStream groupBy( source, resolveSchema(source), rekeyedKeyFormat, - KeyField.of(newKeyCol), ksqlConfig, functionRegistry ); @@ -469,7 +391,6 @@ private SchemaKGroupedStream groupByKey( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -488,7 +409,6 @@ public SchemaKStream flatMap( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry); } @@ -497,10 +417,6 @@ public ExecutionStep getSourceStep() { return sourceStep; } - public KeyField getKeyField() { - return keyField; - } - public LogicalSchema getSchema() { return schema; } @@ -513,20 +429,6 @@ public FunctionRegistry getFunctionRegistry() { return functionRegistry; } - static ColumnName groupedKeyNameFor(final List groupByExpressions) { - if (groupByExpressions.size() == 1 - && groupByExpressions.get(0) instanceof UnqualifiedColumnReferenceExp) { - return ((UnqualifiedColumnReferenceExp) groupByExpressions.get(0)).getColumnName(); - } - - // this is safe because if we group by multiple fields the original field - // will never be in the original schema, so we're necessarily creating a - // new field - return ColumnName.of(groupByExpressions.stream() - .map(Expression::toString) - .collect(Collectors.joining(GROUP_BY_COLUMN_SEPARATOR))); - } - LogicalSchema resolveSchema(final ExecutionStep step) { return new StepSchemaResolver(ksqlConfig, functionRegistry).resolve(step, schema); } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java index 37ba8c29c1b3..67796b81eb11 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java @@ -34,9 +34,7 @@ import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.function.FunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.serde.SerdeOption; @@ -56,7 +54,6 @@ public SchemaKTable( final ExecutionStep> sourceTableStep, final LogicalSchema schema, final KeyFormat keyFormat, - final KeyField keyField, final KsqlConfig ksqlConfig, final FunctionRegistry functionRegistry ) { @@ -64,7 +61,6 @@ public SchemaKTable( null, schema, keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -90,7 +86,6 @@ public SchemaKTable into( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -111,7 +106,6 @@ public SchemaKTable filter( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -123,7 +117,6 @@ public SchemaKTable select( final QueryContext.Stacker contextStacker, final KsqlQueryBuilder ksqlQueryBuilder ) { - final KeyField keyField = findKeyField(selectExpressions); final TableSelect step = ExecutionStepFactory.tableMapValues( contextStacker, sourceTableStep, @@ -134,7 +127,6 @@ public SchemaKTable select( step, resolveSchema(step), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -173,11 +165,6 @@ public SchemaKGroupedTable groupBy( ) { final KeyFormat groupedKeyFormat = KeyFormat.nonWindowed(keyFormat.getFormatInfo()); - final ColumnName aggregateKeyName = groupedKeyNameFor(groupByExpressions); - final Optional newKeyField = getSchema() - .findValueColumn(aggregateKeyName) - .map(Column::name); - final TableGroupBy step = ExecutionStepFactory.tableGroupBy( contextStacker, sourceTableStep, @@ -189,7 +176,6 @@ public SchemaKGroupedTable groupBy( step, resolveSchema(step), groupedKeyFormat, - KeyField.of(newKeyField), ksqlConfig, functionRegistry); } @@ -197,7 +183,6 @@ public SchemaKGroupedTable groupBy( public SchemaKTable join( final SchemaKTable schemaKTable, final ColumnName keyColName, - final KeyField keyField, final Stacker contextStacker ) { final TableTableJoin step = ExecutionStepFactory.tableTableJoin( @@ -211,7 +196,6 @@ public SchemaKTable join( step, resolveSchema(step, schemaKTable), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -220,7 +204,6 @@ public SchemaKTable join( public SchemaKTable leftJoin( final SchemaKTable schemaKTable, final ColumnName keyColName, - final KeyField keyField, final Stacker contextStacker ) { final TableTableJoin step = ExecutionStepFactory.tableTableJoin( @@ -234,7 +217,6 @@ public SchemaKTable leftJoin( step, resolveSchema(step, schemaKTable), keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -243,7 +225,6 @@ public SchemaKTable leftJoin( public SchemaKTable outerJoin( final SchemaKTable schemaKTable, final ColumnName keyColName, - final KeyField keyField, final QueryContext.Stacker contextStacker ) { final TableTableJoin step = ExecutionStepFactory.tableTableJoin( @@ -257,7 +238,6 @@ public SchemaKTable outerJoin( step, resolveSchema(step, schemaKTable), keyFormat, - keyField, ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java index a6c5b49ba27c..b2a0cccd0dd5 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java @@ -21,7 +21,6 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThrows; -import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -29,12 +28,10 @@ import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.MutableMetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; -import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties; import io.confluent.ksql.parser.tree.CreateStreamAsSelect; import io.confluent.ksql.parser.tree.Query; import io.confluent.ksql.parser.tree.Sink; @@ -100,9 +97,6 @@ public void init() { DEFAULT_SERDE_OPTIONS ); - when(sink.getName()).thenReturn(SourceName.of("TEST0")); - when(sink.getProperties()).thenReturn(CreateSourceAsProperties.none()); - query = parseSingle("Select COL0, COL1 from TEST1;"); registerKafkaSource(); @@ -180,7 +174,6 @@ public void shouldNotInheritNamespaceExplicitlySetUpstreamForAvro() { SourceName.of("S0"), schema, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, ksqlTopic @@ -280,7 +273,7 @@ public void shouldThrowOnSelfJoin() { // Given: final CreateStreamAsSelect createStreamAsSelect = parseSingle( "CREATE STREAM FOO AS " - + "SELECT * FROM test1 t1 JOIN test1 t2 ON t1.rowkey = t2.rowkey;" + + "SELECT * FROM test1 t1 JOIN test1 t2 ON t1.col0 = t2.col0;" ); final Query query = createStreamAsSelect.getQuery(); @@ -303,7 +296,7 @@ public void shouldFailOnJoinWithoutSource() { // Given: final CreateStreamAsSelect createStreamAsSelect = parseSingle( "CREATE STREAM FOO AS " - + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.rowkey = 'foo';" + + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.col0 = 'foo';" ); final Query query = createStreamAsSelect.getQuery(); @@ -319,7 +312,7 @@ public void shouldFailOnJoinWithoutSource() { // Then: assertThat(e.getMessage(), containsString( "Invalid comparison expression ''foo'' in join " - + "'(T1.ROWKEY = 'foo')'. Each side of the join comparision must contain references " + + "'(T1.COL0 = 'foo')'. Each side of the join comparision must contain references " + "from exactly one source.")); } @@ -328,7 +321,7 @@ public void shouldFailOnJoinOnOverlappingSources() { // Given: final CreateStreamAsSelect createStreamAsSelect = parseSingle( "CREATE STREAM FOO AS " - + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.rowkey + t2.rowkey = t1.rowkey;" + + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.col0 + t2.col0 = t1.col0;" ); final Query query = createStreamAsSelect.getQuery(); @@ -343,8 +336,8 @@ public void shouldFailOnJoinOnOverlappingSources() { // Then: assertThat(e.getMessage(), containsString( - "Invalid comparison expression '(T1.ROWKEY + T2.ROWKEY)' in " - + "join '((T1.ROWKEY + T2.ROWKEY) = T1.ROWKEY)'. Each side of the join comparision must " + "Invalid comparison expression '(T1.COL0 + T2.COL0)' in " + + "join '((T1.COL0 + T2.COL0) = T1.COL0)'. Each side of the join comparision must " + "contain references from exactly one source.")); } @@ -353,7 +346,7 @@ public void shouldFailOnSelfJoinInCondition() { // Given: final CreateStreamAsSelect createStreamAsSelect = parseSingle( "CREATE STREAM FOO AS " - + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.rowkey = t1.rowkey;" + + "SELECT * FROM test1 t1 JOIN test2 t2 ON t1.col0 = t1.col0;" ); final Query query = createStreamAsSelect.getQuery(); @@ -394,7 +387,6 @@ private void registerKafkaSource() { SourceName.of("KAFKA_SOURCE"), schema, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, topic diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java index 0c8272fc5b1f..f2d78e1a8d5d 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java @@ -62,7 +62,6 @@ import io.confluent.ksql.parser.tree.TableElement.Namespace; import io.confluent.ksql.parser.tree.TableElements; import io.confluent.ksql.properties.with.CommonCreateConfigs; -import io.confluent.ksql.properties.with.CreateConfigs; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.schema.ksql.SystemColumns; @@ -413,24 +412,6 @@ public void shouldNotThrowIfTopicDoesExist() { verify(topicClient).isTopicExists(TOPIC_NAME); } - @Test - public void shouldThrowIfKeyFieldNotInSchemaForStream() { - // Given: - givenProperty(CreateConfigs.KEY_NAME_PROPERTY, new StringLiteral("`will-not-find-me`")); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> createSourceFactory.createStreamCommand(statement, ksqlConfig) - ); - - // Then: - assertThat(e.getMessage(), containsString( - "The KEY column set in the WITH clause does not exist in the schema: " - + "'will-not-find-me'")); - } - @Test public void shouldThrowIfTimestampColumnDoesNotExistForStream() { // Given: diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java index 153a5b8ae398..07273fbbc9a9 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java @@ -1,6 +1,5 @@ package io.confluent.ksql.ddl.commands; -import static io.confluent.ksql.metastore.model.MetaStoreMatchers.KeyFieldMatchers.hasName; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -73,7 +72,6 @@ public class DdlCommandExecTest { @Before public void setup() { - //when(metaStore.getSource(STREAM_NAME)).thenReturn(source); when(source.getName()).thenReturn(STREAM_NAME); when(source.getDataSourceType()).thenReturn(DataSourceType.KSTREAM); when(source.getKafkaTopicName()).thenReturn(TOPIC_NAME); @@ -82,22 +80,10 @@ public void setup() { dropType = new DropTypeCommand("type"); } - @Test - public void shouldAddStreamWithKeyField() { - // Given: - givenCreateStreamWithKey(Optional.of("F1")); - - // When: - cmdExec.execute(SQL_TEXT, createStream, false); - - // Then: - assertThat(metaStore.getSource(STREAM_NAME).getKeyField(), hasName("F1")); - } - @Test public void shouldAddStreamWithCorrectSql() { // Given: - givenCreateStreamWithKey(Optional.of("F1")); + givenCreateStream(); // When: cmdExec.execute(SQL_TEXT, createStream, false); @@ -109,7 +95,7 @@ public void shouldAddStreamWithCorrectSql() { @Test public void shouldAddSinkStream() { // Given: - givenCreateStreamWithKey(Optional.empty()); + givenCreateStream(); // When: cmdExec.execute(SQL_TEXT, createStream, true); @@ -118,22 +104,10 @@ public void shouldAddSinkStream() { assertThat(metaStore.getSource(STREAM_NAME).isCasTarget(), is(true)); } - @Test - public void shouldAddStreamWithNoKeyField() { - // Given: - givenCreateStreamWithKey(Optional.empty()); - - // When: - cmdExec.execute(SQL_TEXT, createStream, false); - - // Then: - assertThat(metaStore.getSource(STREAM_NAME).getKeyField(), hasName(Optional.empty())); - } - @Test public void shouldAddStreamWithCorrectKsqlTopic() { // Given: - givenCreateStreamWithKey(Optional.empty()); + givenCreateStream(); // When: cmdExec.execute(SQL_TEXT, createStream, false); @@ -160,30 +134,6 @@ public void shouldAddStreamWithCorrectWindowInfo() { ); } - @Test - public void shouldAddTableWithKeyField() { - // Given: - givenCreateTableWithKey(Optional.of("F1")); - - // When: - cmdExec.execute(SQL_TEXT, createTable, false); - - // Then: - assertThat(metaStore.getSource(TABLE_NAME).getKeyField(), hasName("F1")); - } - - @Test - public void shouldAddTableWithNoKeyField() { - // Given: - givenCreateTableWithKey(Optional.empty()); - - // When: - cmdExec.execute(SQL_TEXT, createTable, false); - - // Then: - assertThat(metaStore.getSource(TABLE_NAME).getKeyField(), hasName(Optional.empty())); - } - @Test public void shouldAddTableWithCorrectWindowInfo() { // Given: @@ -202,7 +152,7 @@ public void shouldAddTableWithCorrectWindowInfo() { @Test public void shouldAddTableWithCorrectSql() { // Given: - givenCreateTableWithKey(Optional.empty()); + givenCreateTable(); // When: cmdExec.execute(SQL_TEXT, createTable, false); @@ -214,7 +164,7 @@ public void shouldAddTableWithCorrectSql() { @Test public void shouldAddTableWithCorrectTopic() { // Given: - givenCreateTableWithKey(Optional.empty()); + givenCreateTable(); // When: cmdExec.execute(SQL_TEXT, createTable, false); @@ -229,7 +179,7 @@ public void shouldAddTableWithCorrectTopic() { @Test public void shouldAddSinkTable() { // Given: - givenCreateTableWithKey(Optional.empty()); + givenCreateTable(); // When: cmdExec.execute(SQL_TEXT, createTable, true); @@ -299,11 +249,10 @@ private void givenDropSourceCommand(final SourceName name) { dropSource = new DropSourceCommand(name); } - private void givenCreateStreamWithKey(final Optional keyField) { + private void givenCreateStream() { createStream = new CreateStreamCommand( STREAM_NAME, SCHEMA, - keyField.map(ColumnName::of), Optional.of(timestampColumn), "topic", io.confluent.ksql.execution.plan.Formats.of( @@ -318,7 +267,6 @@ private void givenCreateWindowedStream() { createStream = new CreateStreamCommand( STREAM_NAME, SCHEMA, - Optional.empty(), Optional.of(timestampColumn), "topic", io.confluent.ksql.execution.plan.Formats.of( @@ -333,7 +281,6 @@ private void givenCreateWindowedTable() { createTable = new CreateTableCommand( TABLE_NAME, SCHEMA, - Optional.empty(), Optional.of(timestampColumn), TOPIC_NAME, io.confluent.ksql.execution.plan.Formats.of( @@ -345,11 +292,10 @@ private void givenCreateWindowedTable() { ); } - private void givenCreateTableWithKey(final Optional keyField) { + private void givenCreateTable() { createTable = new CreateTableCommand( TABLE_NAME, SCHEMA, - keyField.map(ColumnName::of), Optional.of(timestampColumn), TOPIC_NAME, io.confluent.ksql.execution.plan.Formats.of( diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java index 964be7be27b5..e0ade1ec50f3 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java @@ -47,7 +47,6 @@ import io.confluent.ksql.logging.processing.NoopProcessingLogContext; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.metastore.model.KsqlTable; import io.confluent.ksql.name.ColumnName; @@ -82,7 +81,6 @@ import java.util.concurrent.Future; import java.util.function.LongSupplier; import java.util.function.Supplier; -import java.util.stream.Collectors; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.SerializationException; @@ -97,6 +95,7 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +@SuppressWarnings("UnstableApiUsage") @RunWith(MockitoJUnitRunner.class) public class InsertValuesExecutorTest { @@ -174,7 +173,7 @@ public void setup() { when(serviceContext.getKafkaClientSupplier()).thenReturn(kafkaClientSupplier); when(serviceContext.getSchemaRegistryClientFactory()).thenReturn(srClientFactory); - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none()); when(valueSerdeFactory.create(any(), any(), any(), any(), any(), any())) .thenReturn(valueSerde); @@ -191,8 +190,9 @@ public void setup() { public void shouldHandleFullRow() { // Given: final ConfiguredStatement statement = givenInsertValues( - valueColumnNames(SCHEMA), + allColumnNames(SCHEMA), ImmutableList.of( + new StringLiteral("key"), new StringLiteral("str"), new LongLiteral(2L) ) @@ -202,7 +202,7 @@ public void shouldHandleFullRow() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct("key")); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -210,7 +210,7 @@ public void shouldHandleFullRow() { @Test public void shouldInsertWrappedSingleField() { // Given: - givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( valueColumnNames(SINGLE_VALUE_COLUMN_SCHEMA), @@ -221,7 +221,7 @@ public void shouldInsertWrappedSingleField() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("new")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct(null)); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("new")); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -231,21 +231,20 @@ public void shouldInsertUnwrappedSingleField() { // Given: givenSourceStreamWithSchema( SINGLE_VALUE_COLUMN_SCHEMA, - SerdeOption.of(SerdeOption.UNWRAP_SINGLE_VALUES), - Optional.of(COL0)) - ; + SerdeOption.of(SerdeOption.UNWRAP_SINGLE_VALUES) + ); final ConfiguredStatement statement = givenInsertValues( - valueColumnNames(SINGLE_VALUE_COLUMN_SCHEMA), - ImmutableList.of(new StringLiteral("new")) + allColumnNames(SINGLE_VALUE_COLUMN_SCHEMA), + ImmutableList.of(new StringLiteral("newKey"), new StringLiteral("newCol0")) ); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("new")); - verify(valueSerializer).serialize(TOPIC_NAME, genericRow("new")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct("newKey")); + verify(valueSerializer).serialize(TOPIC_NAME, genericRow("newCol0")); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -286,31 +285,11 @@ public void shouldHandleRowTimeWithoutKey() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct(null)); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1234L, KEY, VALUE)); } - @Test - public void shouldFillInKeyColumnFromSpecifiedKeyField() { - // Given: - final ConfiguredStatement statement = givenInsertValues( - ImmutableList.of(COL0, COL1), - ImmutableList.of( - new StringLiteral("str"), - new LongLiteral(2L) - ) - ); - - // When: - executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); - - // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); - verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); - verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); - } - @Test public void shouldFillInFullRowWithNoSchema() { // Given: @@ -351,34 +330,15 @@ public void shouldFillInMissingColumnsWithNulls() { verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } - @Test - public void shouldFillInKeyFromRowKey() { - // Given: - final ConfiguredStatement statement = givenInsertValues( - ImmutableList.of(K0, COL1), - ImmutableList.of( - new StringLiteral("str"), - new LongLiteral(2L) - ) - ); - - // When: - executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); - - // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); - verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); - verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); - } - @Test public void shouldHandleOutOfOrderSchema() { // Given: final ConfiguredStatement statement = givenInsertValues( - ImmutableList.of(COL1, COL0), + ImmutableList.of(COL1, COL0, K0), ImmutableList.of( new LongLiteral(2L), - new StringLiteral("str") + new StringLiteral("str"), + new StringLiteral("key") ) ); @@ -386,7 +346,7 @@ public void shouldHandleOutOfOrderSchema() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct("key")); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -405,7 +365,6 @@ public void shouldHandleAllSortsOfLiterals() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 2L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -413,10 +372,10 @@ public void shouldHandleAllSortsOfLiterals() { @Test public void shouldHandleNullKeyForSourceWithKeyField() { // Given: - givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( - allColumnNames(BIG_SCHEMA), + allAndPseudoColumnNames(BIG_SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -445,7 +404,7 @@ public void shouldHandleNullKeyForSourceWithKeyField() { @Test public void shouldHandleNegativeValueExpression() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0, COL1), @@ -459,7 +418,7 @@ public void shouldHandleNegativeValueExpression() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); + verify(keySerializer).serialize(TOPIC_NAME, keyStruct(null)); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", -1L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -467,7 +426,7 @@ public void shouldHandleNegativeValueExpression() { @Test public void shouldHandleUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0), @@ -488,7 +447,7 @@ public void shouldHandleUdfs() { @Test public void shouldHandleNestedUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0), @@ -515,7 +474,7 @@ public void shouldHandleNestedUdfs() { @Test public void shouldAllowUpcast() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0, COL1), @@ -529,7 +488,6 @@ public void shouldAllowUpcast() { executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); // Then: - verify(keySerializer).serialize(TOPIC_NAME, keyStruct("str")); verify(valueSerializer).serialize(TOPIC_NAME, genericRow("str", 1L)); verify(producer).send(new ProducerRecord<>(TOPIC_NAME, null, 1L, KEY, VALUE)); } @@ -538,13 +496,13 @@ public void shouldAllowUpcast() { public void shouldThrowWhenInsertValuesOnReservedInternalTopic() { // Given givenDataSourceWithSchema("_confluent-ksql-default__command-topic", SCHEMA, - SerdeOption.none(), Optional.of(COL0), false); + SerdeOption.none(), false); final ConfiguredStatement statement = ConfiguredStatement.of( PreparedStatement.of( "", new InsertValues(SourceName.of("TOPIC"), - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -570,13 +528,13 @@ public void shouldThrowWhenInsertValuesOnReservedInternalTopic() { public void shouldThrowWhenInsertValuesOnProcessingLogTopic() { // Given givenDataSourceWithSchema("default_ksql_processing_log", SCHEMA, - SerdeOption.none(), Optional.of(COL0), false); + SerdeOption.none(), false); final ConfiguredStatement statement = ConfiguredStatement.of( PreparedStatement.of( "", new InsertValues(SourceName.of("TOPIC"), - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -602,7 +560,7 @@ public void shouldThrowWhenInsertValuesOnProcessingLogTopic() { public void shouldThrowOnProducerSendError() throws ExecutionException, InterruptedException { // Given: final ConfiguredStatement statement = givenInsertValues( - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -630,7 +588,7 @@ public void shouldThrowOnProducerSendError() throws ExecutionException, Interrup public void shouldThrowOnSerializingKeyError() { // Given: final ConfiguredStatement statement = givenInsertValues( - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -653,7 +611,7 @@ public void shouldThrowOnSerializingKeyError() { public void shouldThrowOnSerializingValueError() { // Given: final ConfiguredStatement statement = givenInsertValues( - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -677,7 +635,7 @@ public void shouldThrowOnSerializingValueError() { public void shouldThrowOnTopicAuthorizationException() { // Given: final ConfiguredStatement statement = givenInsertValues( - allColumnNames(SCHEMA), + allAndPseudoColumnNames(SCHEMA), ImmutableList.of( new LongLiteral(1L), new StringLiteral("str"), @@ -698,26 +656,6 @@ public void shouldThrowOnTopicAuthorizationException() { containsString("Authorization denied to Write on topic(s): [t1]")))); } - @Test - public void shouldThrowIfRowKeyAndKeyDoNotMatch() { - // Given: - final ConfiguredStatement statement = givenInsertValues( - ImmutableList.of(K0, COL0), - ImmutableList.of( - new StringLiteral("foo"), - new StringLiteral("bar")) - ); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> executor.execute(statement, mock(SessionProperties.class), engine, serviceContext) - ); - - // Then: - assertThat(e.getCause(), (hasMessage(containsString("Expected k0 and COL0 to match")))); - } - @Test public void shouldThrowIfNotEnoughValuesSuppliedWithNoSchema() { // Given: @@ -740,7 +678,7 @@ public void shouldThrowIfNotEnoughValuesSuppliedWithNoSchema() { @Test public void shouldThrowIfColumnDoesNotExistInSchema() { // Given: - givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of( @@ -764,7 +702,7 @@ public void shouldThrowIfColumnDoesNotExistInSchema() { @Test public void shouldFailOnDowncast() { // Given: - givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(INT_COL), @@ -786,7 +724,7 @@ public void shouldFailOnDowncast() { @Test public void shouldHandleStreamsWithNoKeyField() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(K0, COL0, COL1), @@ -808,7 +746,7 @@ public void shouldHandleStreamsWithNoKeyField() { @Test public void shouldHandleTablesWithNoKeyField() { // Given: - givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); + givenSourceTableWithSchema(SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(K0, COL0, COL1), @@ -830,7 +768,7 @@ public void shouldHandleTablesWithNoKeyField() { @Test public void shouldHandleStreamsWithNoKeyFieldAndNoRowKeyProvided() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0, COL1), @@ -851,7 +789,7 @@ public void shouldHandleStreamsWithNoKeyFieldAndNoRowKeyProvided() { @Test public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { // Given: - givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); + givenSourceTableWithSchema(SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL0, COL1), @@ -874,7 +812,7 @@ public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { @Test public void shouldThrowOnTablesWithKeyFieldAndNullKeyFieldValueProvided() { // Given: - givenSourceTableWithSchema(SerdeOption.none(), Optional.of(COL0)); + givenSourceTableWithSchema(SerdeOption.none()); final ConfiguredStatement statement = givenInsertValues( ImmutableList.of(COL1), @@ -942,24 +880,21 @@ private static ConfiguredStatement givenInsertValues( private void givenSourceStreamWithSchema( final LogicalSchema schema, - final Set serdeOptions, - final Optional keyField + final Set serdeOptions ) { - givenDataSourceWithSchema(TOPIC_NAME, schema, serdeOptions, keyField, false); + givenDataSourceWithSchema(TOPIC_NAME, schema, serdeOptions, false); } private void givenSourceTableWithSchema( - final Set serdeOptions, - final Optional keyField + final Set serdeOptions ) { - givenDataSourceWithSchema(TOPIC_NAME, SCHEMA, serdeOptions, keyField, true); + givenDataSourceWithSchema(TOPIC_NAME, SCHEMA, serdeOptions, true); } private void givenDataSourceWithSchema( final String topicName, final LogicalSchema schema, final Set serdeOptions, - final Optional keyField, final boolean table ) { final KsqlTopic topic = new KsqlTopic( @@ -968,10 +903,6 @@ private void givenDataSourceWithSchema( ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())) ); - final KeyField valueKeyField = keyField - .map(KeyField::of) - .orElse(KeyField.none()); - final DataSource dataSource; if (table) { dataSource = new KsqlTable<>( @@ -979,7 +910,6 @@ private void givenDataSourceWithSchema( SourceName.of("TOPIC"), schema, serdeOptions, - valueKeyField, Optional.empty(), false, topic @@ -990,7 +920,6 @@ private void givenDataSourceWithSchema( SourceName.of("TOPIC"), schema, serdeOptions, - valueKeyField, Optional.empty(), false, topic @@ -1010,10 +939,18 @@ private static Struct keyStruct(final String rowKey) { } private static List valueColumnNames(final LogicalSchema schema) { - return schema.value().stream().map(Column::name).collect(Collectors.toList()); + return schema.value().stream() + .map(Column::name) + .collect(ImmutableList.toImmutableList()); } private static List allColumnNames(final LogicalSchema schema) { + return schema.columns().stream() + .map(Column::name) + .collect(ImmutableList.toImmutableList()); + } + + private static List allAndPseudoColumnNames(final LogicalSchema schema) { final Builder builder = ImmutableList.builder() .add(SystemColumns.ROWTIME_NAME); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java index c760f65b431c..afd1a6540822 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java @@ -249,7 +249,7 @@ public void shouldThrowOnInsertIntoStreamWithTableResult() { KsqlEngineTestUtil.execute( serviceContext, ksqlEngine, - "create stream bar as select ROWKEY, itemid, orderid from orders;", + "create stream bar as select ordertime, itemid, orderid from orders;", KSQL_CONFIG, Collections.emptyMap() ); @@ -299,7 +299,7 @@ public void shouldThrowOnInsertIntoWithKeyMismatch() { // Then: assertThat(e, rawMessage(containsString("Incompatible schema between results and sink."))); assertThat(e, rawMessage(containsString("Result schema is `ORDERID` BIGINT KEY, "))); - assertThat(e, rawMessage(containsString("Sink schema is `ROWKEY` BIGINT KEY, "))); + assertThat(e, rawMessage(containsString("Sink schema is `ORDERTIME` BIGINT KEY, "))); assertThat(e, statementText(is( "insert into bar select * from orders partition by orderid;"))); @@ -322,7 +322,7 @@ public void shouldThrowWhenInsertIntoSchemaDoesNotMatch() { () -> execute( serviceContext, ksqlEngine, - "insert into bar select rowkey, itemid from orders;", + "insert into bar select orderTime, itemid from orders;", KSQL_CONFIG, emptyMap() ) @@ -333,7 +333,7 @@ public void shouldThrowWhenInsertIntoSchemaDoesNotMatch() { containsString( "Incompatible schema between results and sink."))); assertThat(e, statementText( - is("insert into bar select rowkey, itemid from orders;"))); + is("insert into bar select orderTime, itemid from orders;"))); } @Test @@ -619,7 +619,6 @@ public void shouldFailIfAvroSchemaNotEvolvable() { "\"name\":\"KsqlDataSourceSchema\"," + "\"namespace\":\"io.confluent.ksql.avro_schemas\"," + "\"fields\":[" + - "{\"name\":\"COL0\",\"type\":[\"null\",\"long\"],\"default\":null}," + "{\"name\":\"COL1\",\"type\":[\"null\",\"string\"],\"default\":null}," + "{\"name\":\"COL2\",\"type\":[\"null\",\"string\"],\"default\":null}," + "{\"name\":\"COL3\",\"type\":[\"null\",\"double\"],\"default\":null}," + @@ -856,8 +855,8 @@ public void shouldHandleMultipleStatements() { + "CREATE STREAM S0 (a INT, b VARCHAR) " + " WITH (kafka_topic='s0_topic', value_format='DELIMITED');\n" + "\n" - + "CREATE TABLE T1 (ROWKEY BIGINT PRIMARY KEY, f0 BIGINT, f1 DOUBLE) " - + " WITH (kafka_topic='t1_topic', value_format='JSON', key = 'f0');\n" + + "CREATE TABLE T1 (f0 BIGINT PRIMARY KEY, f1 DOUBLE) " + + " WITH (kafka_topic='t1_topic', value_format='JSON');\n" + "\n" + "CREATE STREAM S1 AS SELECT * FROM S0;\n" + "\n" diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/EndToEndIntegrationTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/EndToEndIntegrationTest.java index 7ca4053dca74..79a9952fda09 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/EndToEndIntegrationTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/EndToEndIntegrationTest.java @@ -24,7 +24,6 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.startsWith; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.GenericRow; @@ -40,7 +39,6 @@ import io.confluent.ksql.util.TransientQueryMetadata; import io.confluent.ksql.util.UserDataProvider; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -149,11 +147,11 @@ public void before() { ); ksqlContext.sql("CREATE TABLE " + USER_TABLE - + " (registertime bigint, gender varchar, regionid varchar, userid varchar)" - + " WITH (kafka_topic='" + USERS_TOPIC + "', value_format='JSON', key = 'userid');"); + + " (userid varchar PRIMARY KEY, registertime bigint, gender varchar, regionid varchar)" + + " WITH (kafka_topic='" + USERS_TOPIC + "', value_format='JSON');"); ksqlContext.sql("CREATE STREAM " + PAGE_VIEW_STREAM - + " (viewtime bigint, userid varchar, pageid varchar) " + + " (pageid varchar KEY, viewtime bigint, userid varchar) " + "WITH (kafka_topic='" + PAGE_VIEW_TOPIC + "', value_format='JSON');"); } @@ -172,7 +170,7 @@ public void shouldSelectAllFromUsers() throws Exception { final Set actualUsers = rows.stream() .filter(Objects::nonNull) - .peek(row -> assertThat(row.values(), hasSize(5))) + .peek(row -> assertThat(row.values(), hasSize(4))) .map(row -> row.get(0)) .collect(Collectors.toSet()); @@ -180,127 +178,6 @@ public void shouldSelectAllFromUsers() throws Exception { assertThat(actualUsers, is(expectedUsers)); } - @Test - public void shouldSelectFromPageViewsWithSpecificColumn() throws Exception { - final TransientQueryMetadata queryMetadata = - executeStatement("SELECT pageid from %s EMIT CHANGES;", PAGE_VIEW_STREAM); - - final List expectedPages = - Arrays.asList("PAGE_1", "PAGE_2", "PAGE_3", "PAGE_4", "PAGE_5", "PAGE_5", "PAGE_5"); - - final List rows = verifyAvailableRows(queryMetadata, expectedPages.size()); - - final List actualPages = rows.stream() - .filter(Objects::nonNull) - .peek(row -> assertThat(row.values(), hasSize(1))) - .map(row -> row.get(0)) - .collect(Collectors.toList()); - - assertThat(actualPages.subList(0, expectedPages.size()), is(expectedPages)); - assertThat(CONSUMED_COUNT.get(), greaterThan(0)); - } - - @Test - public void shouldSelectAllFromDerivedStream() throws Exception { - executeStatement( - "CREATE STREAM pageviews_female" - + " AS SELECT %s.userid, pageid, regionid, gender " - + " FROM %s " - + " LEFT JOIN %s ON %s.userid = %s.userid" - + " WHERE gender = 'FEMALE';", - PAGE_VIEW_STREAM, PAGE_VIEW_STREAM, USER_TABLE, PAGE_VIEW_STREAM, USER_TABLE); - - final TransientQueryMetadata queryMetadata = executeStatement( - "SELECT * from pageviews_female EMIT CHANGES;"); - - final List results = new ArrayList<>(); - final BlockingRowQueue rowQueue = queryMetadata.getRowQueue(); - - // From the mock data, we expect exactly 3 page views from female users. - final List expectedPages = ImmutableList.of("PAGE_2", "PAGE_5", "PAGE_5"); - final List expectedUsers = ImmutableList.of("USER_2", "USER_0", "USER_2"); - - TestUtils.waitForCondition(() -> { - try { - log.debug("polling from pageviews_female"); - final GenericRow nextRow = rowQueue.poll(1, TimeUnit.SECONDS); - if (nextRow != null) { - results.add(nextRow); - } else { - // If we didn't receive any records on the output topic for 8 seconds, it probably means that the join - // failed because the table data wasn't populated when the stream data was consumed. We should just - // re populate the stream data to try the join again. - log.warn("repopulating {} because the join returned no results.", PAGE_VIEW_TOPIC); - TEST_HARNESS.produceRows( - PAGE_VIEW_TOPIC, PAGE_VIEW_DATA_PROVIDER, JSON, System::currentTimeMillis); - } - } catch (final Exception e) { - throw new RuntimeException("Got exception when polling from pageviews_female", e); - } - return expectedPages.size() <= results.size(); - }, 30000, "Could not consume any records from " + PAGE_VIEW_TOPIC + " for 30 seconds"); - - final List actualPages = new ArrayList<>(); - final List actualUsers = new ArrayList<>(); - - for (final GenericRow result : results) { - final List columns = result.values(); - log.debug("pageview join: {}", columns); - - assertThat(columns, hasSize(4)); - - final String user = (String) columns.get(0); - actualUsers.add(user); - - final String page = (String) columns.get(1); - actualPages.add(page); - } - - assertThat(CONSUMED_COUNT.get(), greaterThan(0)); - assertThat(PRODUCED_COUNT.get(), greaterThan(0)); - assertThat(actualPages, is(expectedPages)); - assertThat(actualUsers, is(expectedUsers)); - } - - @Test - public void shouldCreateStreamUsingLikeClause() throws Exception { - - executeStatement( - "CREATE STREAM pageviews_like_p5" - + " WITH (kafka_topic='pageviews_enriched_r0', value_format='DELIMITED')" - + " AS SELECT * FROM %s" - + " WHERE pageId LIKE '%%_5';", - PAGE_VIEW_STREAM); - - final TransientQueryMetadata queryMetadata = - executeStatement("SELECT userid, pageid from pageviews_like_p5 EMIT CHANGES;"); - - final List columns = waitForFirstRow(queryMetadata); - - assertThat(columns.get(1), is("PAGE_5")); - } - - @Test - public void shouldRetainSelectedColumnsInPartitionBy() throws Exception { - - executeStatement( - "CREATE STREAM pageviews_by_viewtime " - + "AS SELECT viewtime, pageid, userid " - + "from %s " - + "partition by viewtime;", - PAGE_VIEW_STREAM); - - final TransientQueryMetadata queryMetadata = executeStatement( - "SELECT * from pageviews_by_viewtime EMIT CHANGES;"); - - final List columns = waitForFirstRow(queryMetadata); - - assertThat(CONSUMED_COUNT.get(), greaterThan(0)); - assertThat(PRODUCED_COUNT.get(), greaterThan(0)); - assertThat(columns.get(1).toString(), startsWith("PAGE_")); - assertThat(columns.get(2).toString(), startsWith("USER_")); - } - @Test public void shouldSupportDroppingAndRecreatingJoinQuery() throws Exception { final String createStreamStatement = format( diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java index 4812fb00b9c1..a97660cd00f4 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; +import com.google.common.collect.Multimap; import io.confluent.kafka.schemaregistry.avro.AvroSchema; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; @@ -43,10 +44,12 @@ import io.confluent.ksql.util.TestDataProvider; import java.time.Duration; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; @@ -163,7 +166,7 @@ public void deleteInternalTopics(String applicationId) { public void produceRecord(final String topicName, final String key, final String data) { kafkaCluster.produceRows( topicName, - Collections.singletonMap(key, data), + Collections.singletonMap(key, data).entrySet(), new StringSerializer(), new StringSerializer(), DEFAULT_TS_SUPPLIER @@ -178,7 +181,7 @@ public void produceRecord(final String topicName, final String key, final String * @param valueFormat the format values should be produced as. * @return the map of produced rows */ - public Map produceRows( + public Multimap produceRows( final String topic, final TestDataProvider dataProvider, final Format valueFormat @@ -200,7 +203,7 @@ public Map produceRows( * @param timestampSupplier supplier of timestamps. * @return the map of produced rows */ - public Map produceRows( + public Multimap produceRows( final String topic, final TestDataProvider dataProvider, final Format valueFormat, @@ -208,7 +211,7 @@ public Map produceRows( ) { return produceRows( topic, - dataProvider.data(), + dataProvider.data().entries(), getKeySerializer(dataProvider.schema()), getValueSerializer(valueFormat, dataProvider.schema()), timestampSupplier @@ -224,9 +227,9 @@ public Map produceRows( * @param valueFormat the format values should be produced as. * @return the map of produced rows */ - public Map produceRows( + public Multimap produceRows( final String topic, - final Map rowsToPublish, + final Collection> rowsToPublish, final PhysicalSchema schema, final Format valueFormat ) { @@ -249,9 +252,9 @@ public Map produceRows( * @param timestampSupplier supplier of timestamps. * @return the map of produced rows, with an iteration order that matches produce order. */ - public Map produceRows( + public Multimap produceRows( final String topic, - final Map recordsToPublish, + final Collection> recordsToPublish, final Serializer keySerializer, final Serializer valueSerializer, final Supplier timestampSupplier @@ -404,7 +407,7 @@ public Map verifyAvailableUniqueRows( final Format valueFormat, final PhysicalSchema schema ) { - return verifyAvailableUniqueRows(topic, is(expectedCount), valueFormat, schema); + return verifyAvailableNumUniqueRows(topic, is(expectedCount), valueFormat, schema); } /** @@ -416,7 +419,7 @@ public Map verifyAvailableUniqueRows( * @param schema the schema of the value. * @return the list of consumed records. */ - public Map verifyAvailableUniqueRows( + public Map verifyAvailableNumUniqueRows( final String topic, final Matcher expectedCount, final Format valueFormat, @@ -427,7 +430,7 @@ public Map verifyAvailableUniqueRows( return verifyAvailableUniqueRows( topic, - expectedCount, + mapHasSize(expectedCount), keyDeserializer, valueDeserializer ); @@ -437,14 +440,40 @@ public Map verifyAvailableUniqueRows( * Verify there are {@code expectedCount} unique rows available on the supplied {@code topic}. * * @param topic the name of the topic to check. - * @param expectedCount the expected number of records. + * @param expected the expected records. + * @param valueFormat the format of the value. + * @param schema the schema of the value. + * @return the list of consumed records. + */ + public Map verifyAvailableUniqueRows( + final String topic, + final Matcher> expected, + final Format valueFormat, + final PhysicalSchema schema + ) { + final Deserializer keyDeserializer = getKeyDeserializer(schema); + final Deserializer valueDeserializer = getValueDeserializer(valueFormat, schema); + + return verifyAvailableUniqueRows( + topic, + expected, + keyDeserializer, + valueDeserializer + ); + } + + /** + * Verify there are {@code expected} unique rows available on the supplied {@code topic}. + * + * @param topic the name of the topic to check. + * @param expected the expected records. * @param keyDeserializer the keyDeserilizer to use. * @param valueDeserializer the valueDeserializer of use. * @return the list of consumed records. */ public Map verifyAvailableUniqueRows( final String topic, - final Matcher expectedCount, + final Matcher> expected, final Deserializer keyDeserializer, final Deserializer valueDeserializer ) { @@ -456,7 +485,7 @@ public Map verifyAvailableUniqueRows( consumer.subscribe(Collections.singleton(topic)); final List> consumerRecords = ConsumerTestUtil - .verifyAvailableRecords(consumer, hasUniqueRecords(mapHasSize(expectedCount))); + .verifyAvailableRecords(consumer, hasUniqueRecords(expected)); return toUniqueRecords(consumerRecords); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JoinIntTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JoinIntTest.java index 2b11bd2b17d4..f0d66c9935e3 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JoinIntTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JoinIntTest.java @@ -274,7 +274,7 @@ public void shouldUseTimeStampFieldFromStream() throws Exception { private void createStreams() { ksqlContext.sql(String.format( - "CREATE STREAM %s (ORDERTIME bigint, ORDERID varchar, " + "CREATE STREAM %s (ORDERID varchar KEY, ORDERTIME bigint, " + "ITEMID varchar, ORDERUNITS double, PRICEARRAY array, " + "KEYVALUEMAP map) " + "WITH (kafka_topic='%s', value_format='JSON');", @@ -282,13 +282,13 @@ private void createStreams() { orderStreamTopicJson)); ksqlContext.sql(String.format( - "CREATE TABLE %s (ID varchar, DESCRIPTION varchar) " - + "WITH (kafka_topic='%s', value_format='JSON', key='ID');", + "CREATE TABLE %s (ID varchar PRIMARY KEY, DESCRIPTION varchar) " + + "WITH (kafka_topic='%s', value_format='JSON');", ITEM_TABLE_NAME_JSON, itemTableTopicJson)); ksqlContext.sql(String.format( - "CREATE STREAM %s (ORDERTIME bigint, ORDERID varchar, ITEMID varchar, " + "CREATE STREAM %s (ORDERID varchar KEY, ORDERTIME bigint, ITEMID varchar, " + "ORDERUNITS double, PRICEARRAY array, " + "KEYVALUEMAP map) " + "WITH (kafka_topic='%s', value_format='AVRO', timestamp='ORDERTIME');", @@ -296,8 +296,8 @@ private void createStreams() { orderStreamTopicAvro)); ksqlContext.sql(String.format( - "CREATE TABLE %s (ID varchar, DESCRIPTION varchar)" - + " WITH (kafka_topic='%s', value_format='AVRO', key='ID');", + "CREATE TABLE %s (ID varchar PRIMARY KEY, DESCRIPTION varchar)" + + " WITH (kafka_topic='%s', value_format='AVRO');", ITEM_TABLE_NAME_AVRO, itemTableTopicAvro)); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java index 9b0e8440666c..367db63dde07 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java @@ -62,15 +62,16 @@ @Category({IntegrationTest.class}) public class JsonFormatTest { + private static final String inputTopic = "orders_topic"; private static final String inputStream = "ORDERS"; private static final String usersTopic = "users_topic"; - private static final String usersTable = "USERS"; private static final String messageLogTopic = "log_topic"; private static final String messageLogStream = "message_log"; private static final AtomicInteger COUNTER = new AtomicInteger(); public static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + private static final OrderDataProvider ORDER_DATA_PROVIDER = new OrderDataProvider(); @ClassRule public static final RuleChain CLUSTER_WITH_RETRY = RuleChain @@ -115,9 +116,7 @@ private void createInitTopics() { } private static void produceInitData() { - final OrderDataProvider orderDataProvider = new OrderDataProvider(); - - TEST_HARNESS.produceRows(inputTopic, orderDataProvider, JSON); + TEST_HARNESS.produceRows(inputTopic, ORDER_DATA_PROVIDER, JSON); final LogicalSchema messageSchema = LogicalSchema.builder() .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) @@ -137,29 +136,21 @@ private static void produceInitData() { SerdeOption.none() ); - TEST_HARNESS.produceRows(messageLogTopic, records, schema, JSON); + TEST_HARNESS.produceRows(messageLogTopic, records.entrySet(), schema, JSON); } private void execInitCreateStreamQueries() { - final String ordersStreamStr = String.format("CREATE STREAM %s (" - + "ROWKEY BIGINT KEY, ORDERTIME bigint, ORDERID varchar, " - + "ITEMID varchar, ORDERUNITS double, PRICEARRAY array, KEYVALUEMAP " - + "map) WITH (value_format = 'json', " - + "kafka_topic='%s' , " - + "key='ordertime');", inputStream, inputTopic); - - final String usersTableStr = String.format("CREATE TABLE %s (userid varchar, age integer) WITH " - + "(value_format = 'json', kafka_topic='%s', " - + "KEY='userid');", - usersTable, usersTopic); + final String ordersStreamStr = "CREATE STREAM " + inputStream + " (" + + ORDER_DATA_PROVIDER.ksqlSchemaString(false) + + ") WITH (value_format = 'json', " + + "kafka_topic='" + inputTopic + "');"; final String messageStreamStr = String.format("CREATE STREAM %s (message varchar) WITH (value_format = 'json', " + "kafka_topic='%s');", messageLogStream, messageLogTopic); KsqlEngineTestUtil.execute( serviceContext, ksqlEngine, ordersStreamStr, ksqlConfig, Collections.emptyMap()); - KsqlEngineTestUtil.execute( - serviceContext, ksqlEngine, usersTableStr, ksqlConfig, Collections.emptyMap()); + KsqlEngineTestUtil.execute( serviceContext, ksqlEngine, messageStreamStr, ksqlConfig, Collections.emptyMap()); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java index 94cc675f2c94..b8c689e5d3d1 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java @@ -75,7 +75,7 @@ public void shouldSelectChosenColumns() { // When: final GenericRow transformed = selectTransformer.transform( NON_WINDOWED_KEY, - genericRow(1L, "hi", "bye", 2.0D, "blah", 1521834663L, "key1"), + genericRow("hi", "bye", 2.0D, "blah", "dar", 1521834663L, 1L), ctx ); @@ -94,7 +94,7 @@ public void shouldApplyUdfsToColumns() { // When: final GenericRow row = selectTransformer.transform( NON_WINDOWED_KEY, - genericRow(2L, "foo", "whatever", 6.9D, "boo", "hoo", 1521834663L, "key1"), + genericRow("foo", "whatever", 6.9D, "boo", "hoo", 1521834663L, 2L), ctx ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java index 04845e614be4..cfeae34cde7c 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java @@ -22,6 +22,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; import io.confluent.common.utils.IntegrationTest; import io.confluent.ksql.GenericRow; import io.confluent.ksql.metastore.model.DataSource; @@ -69,8 +71,8 @@ public class StreamsSelectAndProjectIntTest { private String avroTopicName; private String intermediateStream; private String resultStream; - private Map producedAvroRecords; - private Map producedJsonRecords; + private Multimap producedAvroRecords; + private Multimap producedJsonRecords; @Before public void before() { @@ -162,10 +164,10 @@ public void shouldSkipBadDataAvro() { public void shouldUseStringTimestampWithFormat() throws Exception { ksqlContext.sql("CREATE STREAM " + intermediateStream + " WITH (timestamp='TIMESTAMP', timestamp_format='yyyy-MM-dd') AS" - + " SELECT ROWKEY, ORDERID, TIMESTAMP FROM " + AVRO_STREAM_NAME + " WHERE ITEMID='ITEM_6';" + + " SELECT ORDERTIME, ORDERID, TIMESTAMP FROM " + AVRO_STREAM_NAME + " WHERE ITEMID='ITEM_6';" + "" + " CREATE STREAM " + resultStream + " AS" - + " SELECT ROWKEY, ORDERID, TIMESTAMP from " + intermediateStream + ";"); + + " SELECT ORDERTIME, ORDERID, TIMESTAMP from " + intermediateStream + ";"); final List> records = TEST_HARNESS.verifyAvailableRecords(resultStream.toUpperCase(), 1); @@ -178,7 +180,7 @@ public void shouldUseStringTimestampWithFormat() throws Exception { @Test public void shouldUseTimestampExtractedFromDDLStatement() throws Exception { ksqlContext.sql("CREATE STREAM " + resultStream + " WITH(timestamp='ordertime')" - + " AS SELECT ROWKEY, ORDERID, ORDERTIME FROM " + AVRO_TIMESTAMP_STREAM_NAME + + " AS SELECT ORDERID, ORDERTIME FROM " + AVRO_TIMESTAMP_STREAM_NAME + " WHERE ITEMID='ITEM_4';"); final List> records = @@ -191,31 +193,32 @@ public void shouldUseTimestampExtractedFromDDLStatement() throws Exception { private void testTimestampColumnSelection( final String inputStreamName, final Format dataSourceSerDe, - final Map recordMetadataMap + final Multimap recordMetadataMap ) { final String query1String = - String.format("CREATE STREAM %s WITH (timestamp='RTIME') AS SELECT ROWKEY AS RKEY, " + String.format("CREATE STREAM %s WITH (timestamp='RTIME') AS SELECT ORDERTIME, " + "ROWTIME+10000 AS " + "RTIME, ROWTIME+100 AS RT100, ORDERID, ITEMID " + "FROM %s WHERE ORDERUNITS > 20 AND ITEMID = 'ITEM_8'; " + "" - + "CREATE STREAM %s AS SELECT RKEY AS NEWRKEY, " - + "ROWTIME AS NEWRTIME, AS_VALUE(RKEY) AS RKEY, RTIME, RT100, ORDERID, ITEMID " + + "CREATE STREAM %s AS SELECT ORDERTIME, " + + "ROWTIME AS NEWRTIME, RTIME, RT100, ORDERID, ITEMID " + "FROM %s ;", intermediateStream, inputStreamName, resultStream, intermediateStream); ksqlContext.sql(query1String); final Map expectedResults = new HashMap<>(); + final RecordMetadata order_6 = Iterables.getLast(recordMetadataMap.get("ORDER_6")); expectedResults.put(8L, genericRow( null, null, "8", - recordMetadataMap.get(8L).timestamp() + 10000, + order_6.timestamp() + 10000, "8", - recordMetadataMap.get(8L).timestamp() + 10000, - recordMetadataMap.get(8L).timestamp() + 100, + order_6.timestamp() + 10000, + order_6.timestamp() + 100, "ORDER_6", "ITEM_8") ); @@ -230,11 +233,11 @@ private void testTimestampColumnSelection( private void testSelectProjectKeyTimestamp( final String inputStreamName, final Format valueFormat, - final Map recordMetadataMap + final Multimap recordMetadataMap ) { - ksqlContext.sql(String.format("CREATE STREAM %s AS SELECT ROWKEY AS RKEY, ROWTIME " - + "AS RTIME, ITEMID FROM %s WHERE ORDERUNITS > 20 AND ITEMID = " - + "'ITEM_8';", resultStream, inputStreamName)); + ksqlContext.sql(String.format("CREATE STREAM %s AS SELECT ORDERID, ORDERTIME, ROWTIME AS RTIME, ITEMID " + + "FROM %s WHERE ORDERUNITS > 20 AND ITEMID = " + + "'ITEM_8';", resultStream, inputStreamName)); final List> results = TEST_HARNESS.verifyAvailableRows( resultStream.toUpperCase(), @@ -243,16 +246,21 @@ private void testSelectProjectKeyTimestamp( getResultSchema() ); - assertThat(results.get(0).key(), is(8L)); + assertThat(results.get(0).key(), is("ORDER_6")); assertThat(results.get(0).value(), - is(genericRow(recordMetadataMap.get(8L).timestamp(), "ITEM_8"))); + is(genericRow( + 8L, + Iterables.getLast(recordMetadataMap.get("ORDER_6")).timestamp(), + "ITEM_8") + ) + ); } private void testSelectProject( final String inputStreamName, final Format dataSourceSerDe ) { - ksqlContext.sql(String.format("CREATE STREAM %s AS SELECT ROWKEY, ITEMID, ORDERUNITS, " + ksqlContext.sql(String.format("CREATE STREAM %s AS SELECT ORDERID, ORDERTIME, ITEMID, ORDERUNITS, " + "PRICEARRAY FROM %s;", resultStream, inputStreamName)); final List> results = TEST_HARNESS.verifyAvailableRows( @@ -262,7 +270,7 @@ private void testSelectProject( getResultSchema()); final GenericRow value = results.get(0).value(); - assertThat(value.get(0), is("ITEM_1")); + assertThat(value.get(1), is("ITEM_1")); } @@ -270,7 +278,8 @@ private void testSelectProject( public void testSelectProjectAvroJson() { ksqlContext.sql(String.format("CREATE STREAM %s WITH ( value_format = 'JSON') AS SELECT " - + "ROWKEY, " + + "ORDERID, " + + "ORDERTIME, " + "ITEMID, " + "ORDERUNITS, " + "PRICEARRAY FROM %s;", resultStream, AVRO_STREAM_NAME)); @@ -282,7 +291,7 @@ public void testSelectProjectAvroJson() { getResultSchema()); final GenericRow value = results.get(0).value(); - assertThat(value.get(0), is("ITEM_1")); + assertThat(value.get(1), is("ITEM_1")); } private void testSelectStar( @@ -293,14 +302,12 @@ private void testSelectStar( resultStream, inputStreamName)); - final Map results = TEST_HARNESS.verifyAvailableUniqueRows( + TEST_HARNESS.verifyAvailableUniqueRows( resultStream.toUpperCase(), - DATA_PROVIDER.data().size(), + is(DATA_PROVIDER.finalData()), valueFormat, DATA_PROVIDER.schema() ); - - assertThat(results, is(DATA_PROVIDER.data())); } private void testSelectWithFilter( @@ -319,10 +326,10 @@ private void testSelectWithFilter( @Test public void testInsertIntoJson() { - givenStreamExists(resultStream, "ROWKEY, ITEMID, ORDERUNITS, PRICEARRAY", JSON_STREAM_NAME); + givenStreamExists(resultStream, "ORDERID, ORDERTIME, ITEMID, ORDERUNITS, PRICEARRAY", JSON_STREAM_NAME); ksqlContext.sql("INSERT INTO " + resultStream + - " SELECT ROWKEY, ITEMID, ORDERUNITS, PRICEARRAY FROM " + JSON_STREAM_NAME + ";"); + " SELECT ORDERID, ORDERTIME, ITEMID, ORDERUNITS, PRICEARRAY FROM " + JSON_STREAM_NAME + ";"); final List> results = TEST_HARNESS.verifyAvailableRows( resultStream.toUpperCase(), @@ -331,16 +338,16 @@ public void testInsertIntoJson() { getResultSchema()); final GenericRow value = results.get(0).value(); - assertThat(value.get(0), is("ITEM_1")); + assertThat(value.get(1), is("ITEM_1")); } @Test public void testInsertIntoAvro() { - givenStreamExists(resultStream, "ROWKEY, ITEMID, ORDERUNITS, PRICEARRAY", AVRO_STREAM_NAME); + givenStreamExists(resultStream, "ORDERID, ORDERTIME, ITEMID, ORDERUNITS, PRICEARRAY", AVRO_STREAM_NAME); ksqlContext.sql("INSERT INTO " + resultStream + " " - + "SELECT ROWKEY, ITEMID, ORDERUNITS, PRICEARRAY FROM " + AVRO_STREAM_NAME + ";"); + + "SELECT ORDERID, ORDERTIME, ITEMID, ORDERUNITS, PRICEARRAY FROM " + AVRO_STREAM_NAME + ";"); final List> results = TEST_HARNESS.verifyAvailableRows( resultStream.toUpperCase(), @@ -349,7 +356,7 @@ public void testInsertIntoAvro() { getResultSchema()); final GenericRow value = results.get(0).value(); - assertThat(value.get(0), is("ITEM_1")); + assertThat(value.get(1), is("ITEM_1")); } @Test @@ -359,14 +366,12 @@ public void testInsertSelectStarJson() { ksqlContext.sql("INSERT INTO " + resultStream + " SELECT * FROM " + JSON_STREAM_NAME + ";"); - final Map results = TEST_HARNESS.verifyAvailableUniqueRows( + TEST_HARNESS.verifyAvailableUniqueRows( resultStream.toUpperCase(), - DATA_PROVIDER.data().size(), + is(DATA_PROVIDER.finalData()), JSON, DATA_PROVIDER.schema() ); - - assertThat(results, is(DATA_PROVIDER.data())); } @Test @@ -376,14 +381,12 @@ public void testInsertSelectStarAvro() { ksqlContext.sql("INSERT INTO " + resultStream + " SELECT * FROM " + AVRO_STREAM_NAME + ";"); - final Map results = TEST_HARNESS.verifyAvailableUniqueRows( + TEST_HARNESS.verifyAvailableUniqueRows( resultStream.toUpperCase(), - DATA_PROVIDER.data().size(), + is(DATA_PROVIDER.finalData()), AVRO, DATA_PROVIDER.schema() ); - - assertThat(results, is(DATA_PROVIDER.data())); } @Test diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/UdfIntTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/UdfIntTest.java index 0eb25b2742a0..6c722d3e4a3c 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/UdfIntTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/UdfIntTest.java @@ -25,7 +25,10 @@ import static org.hamcrest.Matchers.not; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; import io.confluent.common.utils.IntegrationTest; import io.confluent.ksql.GenericRow; import io.confluent.ksql.metastore.model.DataSource; @@ -65,8 +68,8 @@ public class UdfIntTest { private static final String DELIMITED_TOPIC_NAME = "delimitedTopic"; private static final String DELIMITED_STREAM_NAME = "items_delimited"; - private static Map jsonRecordMetadataMap; - private static Map avroRecordMetadataMap; + private static Multimap jsonRecordMetadataMap; + private static Multimap avroRecordMetadataMap; private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); @@ -96,11 +99,9 @@ public static void classSetUp() { final OrderDataProvider orderDataProvider = new OrderDataProvider(); final ItemDataProvider itemDataProvider = new ItemDataProvider(); - jsonRecordMetadataMap = ImmutableMap.copyOf( - TEST_HARNESS.produceRows(JSON_TOPIC_NAME, orderDataProvider, JSON)); + jsonRecordMetadataMap = TEST_HARNESS.produceRows(JSON_TOPIC_NAME, orderDataProvider, JSON); - avroRecordMetadataMap = ImmutableMap.copyOf( - TEST_HARNESS.produceRows(AVRO_TOPIC_NAME, orderDataProvider, AVRO)); + avroRecordMetadataMap = TEST_HARNESS.produceRows(AVRO_TOPIC_NAME, orderDataProvider, AVRO); TEST_HARNESS.produceRows(DELIMITED_TOPIC_NAME, itemDataProvider, DELIMITED); } @@ -117,7 +118,7 @@ public UdfIntTest(final Format format) { break; default: this.testData = - new TestData(format, DELIMITED_TOPIC_NAME, DELIMITED_STREAM_NAME, ImmutableMap.of()); + new TestData(format, DELIMITED_TOPIC_NAME, DELIMITED_STREAM_NAME, ImmutableListMultimap.of()); break; } } @@ -137,7 +138,7 @@ public void testApplyUdfsToColumns() { // Given: final String queryString = String.format( "CREATE STREAM \"%s\" AS SELECT " - + "ROWKEY, " + + "ORDERID, " + "ITEMID, " + "ORDERUNITS*10, " + "PRICEARRAY[1]+10, " @@ -154,7 +155,7 @@ public void testApplyUdfsToColumns() { // Then: final Map results = consumeOutputMessages(); - assertThat(results, is(ImmutableMap.of(8L, + assertThat(results, is(ImmutableMap.of("ORDER_6", genericRow("ITEM_8", 800.0, 1110.0, 12.0, true)))); } @@ -165,7 +166,7 @@ public void testShouldCastSelectedColumns() { // Given: final String queryString = String.format( "CREATE STREAM \"%s\" AS SELECT " - + "ROWKEY, " + + "ORDERID, " + "CAST (ORDERUNITS AS INTEGER), " + "CAST( PRICEARRAY[2]>1000 AS STRING), " + "CAST (SUBSTRING(ITEMID, 6) AS DOUBLE), " @@ -181,7 +182,7 @@ public void testShouldCastSelectedColumns() { // Then: final Map results = consumeOutputMessages(); - assertThat(results, is(ImmutableMap.of(8L, + assertThat(results, is(ImmutableMap.of("ORDER_6", genericRow(80, "true", 8.0, "80.0")))); } @@ -192,11 +193,11 @@ public void testTimestampColumnSelection() { // Given: final String queryString = String.format( "CREATE STREAM \"%s\" AS SELECT " - + "ROWKEY AS RKEY, ROWTIME+10000 AS RTIME, ROWTIME+100 AS RT100, ORDERID, ITEMID " + + "ROWTIME+10000 AS RTIME, ROWTIME+100 AS RT100, ORDERID, ITEMID " + "FROM %s WHERE ORDERUNITS > 20 AND ITEMID = 'ITEM_8';" + "" + "CREATE STREAM \"%s\" AS SELECT " - + "RKEY AS NEWRKEY, ROWTIME AS NEWRTIME, AS_VALUE(RKEY) AS RKEY, RTIME, RT100, ORDERID, ITEMID " + + "ROWTIME AS NEWRTIME, RTIME, RT100, ORDERID, ITEMID " + "FROM %s;", intermediateStream, testData.sourceStreamName, resultStreamName, intermediateStream); @@ -204,12 +205,12 @@ public void testTimestampColumnSelection() { ksqlContext.sql(queryString); // Then: - final Map results = consumeOutputMessages(); + final Map results = consumeOutputMessages(); - final long ts = testData.recordMetadata.get(8L).timestamp(); + final long ts = testData.getLast("ORDER_6").timestamp(); - assertThat(results, equalTo(ImmutableMap.of(8L, - genericRow(ts, 8L, ts + 10000, ts + 100, "ORDER_6", "ITEM_8")))); + assertThat(results, equalTo(ImmutableMap.of("ORDER_6", + genericRow(ts, ts + 10000, ts + 100, "ITEM_8")))); } @Test @@ -218,7 +219,7 @@ public void testApplyUdfsToColumnsDelimited() { // Given: final String queryString = String.format( - "CREATE STREAM \"%s\" AS SELECT ROWKEY, ID, DESCRIPTION FROM %s WHERE ID LIKE '%%_1';", + "CREATE STREAM \"%s\" AS SELECT ID, DESCRIPTION FROM %s WHERE ID LIKE '%%_1';", resultStreamName, DELIMITED_STREAM_NAME ); @@ -229,21 +230,20 @@ public void testApplyUdfsToColumnsDelimited() { final Map results = consumeOutputMessages(); assertThat(results, equalTo(Collections.singletonMap("ITEM_1", - genericRow("ITEM_1", "home cinema")))); + genericRow("home cinema")))); } private void createSourceStream() { if (testData.format == DELIMITED) { // Delimited does not support array or map types, so use simplier schema: ksqlContext.sql(String.format("CREATE STREAM %s " - + "(ROWKEY STRING KEY, ID varchar, DESCRIPTION varchar) WITH " + + "(ID varchar KEY, DESCRIPTION varchar) WITH " + "(kafka_topic='%s', value_format='%s');", testData.sourceStreamName, testData.sourceTopicName, testData.format.name())); } else { ksqlContext.sql(String.format("CREATE STREAM %s (" - + "ROWKEY BIGINT KEY, " + + "ORDERID varchar KEY, " + "ORDERTIME bigint, " - + "ORDERID varchar, " + "ITEMID varchar, " + "ORDERUNITS double, " + "TIMESTAMP varchar, " @@ -277,17 +277,21 @@ private static class TestData { private final Format format; private final String sourceStreamName; private final String sourceTopicName; - private final Map recordMetadata; + private final Multimap recordMetadata; private TestData( final Format format, final String sourceTopicName, final String sourceStreamName, - final Map recordMetadata) { + final Multimap recordMetadata) { this.format = format; this.sourceStreamName = sourceStreamName; this.sourceTopicName = sourceTopicName; this.recordMetadata = recordMetadata; } + + RecordMetadata getLast(final String key) { + return Iterables.getLast(recordMetadata.get(key)); + } } } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/WindowingIntTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/WindowingIntTest.java index 058bc635f2d2..10bca064c7b1 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/WindowingIntTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/WindowingIntTest.java @@ -210,7 +210,7 @@ public void shouldAggregateSessionWindow() { // Then: assertOutputOf(resultStream0, expected, mapHasItems(expected)); - assertTopicsCleanedUp(TopicCleanupPolicy.DELETE, 3, resultStream0); + assertTopicsCleanedUp(TopicCleanupPolicy.DELETE, 2, resultStream0); } private void givenTable(final String sql) { @@ -288,8 +288,8 @@ private static Deserializer getKeyDeserializerFor(final Object key) { private void createOrdersStream() { ksqlContext.sql("CREATE STREAM " + ORDERS_STREAM + " (" + + "ORDERID varchar KEY, " + "ORDERTIME bigint, " - + "ORDERID varchar, " + "ITEMID varchar, " + "ORDERUNITS double, " + "PRICEARRAY array, " diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java index 4d329772081b..5d73566b7068 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java @@ -67,27 +67,27 @@ public class PhysicalPlanBuilderTest { + "WITH (KAFKA_TOPIC = 'test1', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST2 = "CREATE STREAM TEST2 " - + "(ROWKEY BIGINT KEY, ID2 BIGINT, COL0 VARCHAR, COL1 BIGINT) " - + " WITH (KAFKA_TOPIC = 'test2', VALUE_FORMAT = 'JSON', KEY='ID2');"; + + "(ID2 BIGINT KEY, COL0 VARCHAR, COL1 BIGINT) " + + " WITH (KAFKA_TOPIC = 'test2', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST3 = "CREATE STREAM TEST3 " - + "(ROWKEY BIGINT KEY, ID3 BIGINT, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test3', VALUE_FORMAT = 'JSON', KEY='ID3');"; + + "(ID3 BIGINT KEY, COL0 BIGINT, COL1 DOUBLE) " + + " WITH (KAFKA_TOPIC = 'test3', VALUE_FORMAT = 'JSON');"; private static final String CREATE_TABLE_TEST4 = "CREATE TABLE TEST4 " - + "(ROWKEY BIGINT PRIMARY KEY, ID BIGINT, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test4', VALUE_FORMAT = 'JSON', KEY='ID');"; + + "(ID BIGINT PRIMARY KEY, COL0 BIGINT, COL1 DOUBLE) " + + " WITH (KAFKA_TOPIC = 'test4', VALUE_FORMAT = 'JSON');"; private static final String CREATE_TABLE_TEST5 = "CREATE TABLE TEST5 " - + "(ROWKEY BIGINT PRIMARY KEY, ID BIGINT, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test5', VALUE_FORMAT = 'JSON', KEY='ID');"; + + "(ID BIGINT PRIMARY KEY, COL0 BIGINT, COL1 DOUBLE) " + + " WITH (KAFKA_TOPIC = 'test5', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST6 = "CREATE STREAM TEST6 " - + "(ID BIGINT, COL0 VARCHAR, COL1 DOUBLE) " + + "(ID BIGINT KEY, COL0 VARCHAR, COL1 DOUBLE) " + " WITH (KAFKA_TOPIC = 'test6', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST7 = "CREATE STREAM TEST7 " - + "(ID BIGINT, COL0 VARCHAR, COL1 DOUBLE) " + + "(ID BIGINT KEY, COL0 VARCHAR, COL1 DOUBLE) " + " WITH (KAFKA_TOPIC = 'test7', VALUE_FORMAT = 'JSON');"; private static final String simpleSelectFilter = "SELECT rowkey, col0, col2 FROM test1 WHERE col0 > 100 EMIT CHANGES;"; @@ -251,7 +251,7 @@ public void shouldRepartitionLeftStreamIfNotCorrectKey() { // Then: assertThat(result.getExecutionPlan(), containsString( - "[ REKEY ] | Schema: COL1 BIGINT KEY, ID2" + "[ REKEY ] | Schema: COL1 BIGINT KEY," )); } @@ -269,7 +269,7 @@ public void shouldRepartitionRightStreamIfNotCorrectKey() { // Then: assertThat(result.getExecutionPlan(), containsString( - "[ REKEY ] | Schema: COL0 BIGINT KEY, ID3" + "[ REKEY ] | Schema: COL0 BIGINT KEY," )); } @@ -290,7 +290,7 @@ public void shouldThrowIfLeftTableNotJoiningOnTableKey() { // Then: assertThat(e.getMessage(), containsString( "Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0")); + + "sure that the criteria uses the TABLE's key column ID instead of COL0")); } @Test @@ -310,7 +310,7 @@ public void shouldThrowIfRightTableNotJoiningOnTableKey() { // Then: assertThat(e.getMessage(), containsString( "Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0")); + + "sure that the criteria uses the TABLE's key column ID instead of COL0")); } @Test @@ -329,66 +329,6 @@ public void shouldNotRepartitionEitherStreamsIfJoiningOnKeys() { assertThat(result.getExecutionPlan(), not(containsString("[ REKEY ]"))); } - @Test - public void shouldNotRepartitionEitherStreamsIfJoiningOnRowKey() { - // Given: - givenKafkaTopicsExist("test2", "test3"); - execute(CREATE_STREAM_TEST2 + CREATE_STREAM_TEST3); - - // When: - final QueryMetadata result = execute("CREATE STREAM s1 AS " - + "SELECT * FROM test2 JOIN test3 WITHIN 1 SECOND " - + "ON test2.rowkey = test3.rowkey;") - .get(0); - - // Then: - assertThat(result.getExecutionPlan(), not(containsString("[ REKEY ]"))); - } - - @Test - public void shouldNotRepartitionEitherStreamsIfJoiningOnRowKeyEvenIfStreamsHaveNoKeyField() { - // Given: - givenKafkaTopicsExist("test6", "test7"); - execute(CREATE_STREAM_TEST6 + CREATE_STREAM_TEST7); - - // When: - final QueryMetadata result = execute("CREATE STREAM s1 AS " - + "SELECT * FROM test6 JOIN test7 WITHIN 1 SECOND " - + "ON test6.rowkey = test7.rowkey;") - .get(0); - - // Then: - assertThat(result.getExecutionPlan(), not(containsString("[ REKEY ]"))); - } - - @Test - public void shouldHandleLeftTableJoiningOnRowKey() { - // Given: - givenKafkaTopicsExist("test4", "test5"); - execute(CREATE_TABLE_TEST4 + CREATE_TABLE_TEST5); - - // When: - execute("CREATE TABLE t1 AS " - + "SELECT * FROM test4 JOIN test5 " - + "ON test4.rowkey = test5.id;"); - - // Then: did not throw. - } - - @Test - public void shouldHandleRightTableJoiningOnRowKey() { - // Given: - givenKafkaTopicsExist("test4", "test5"); - execute(CREATE_TABLE_TEST4 + CREATE_TABLE_TEST5); - - // When: - execute("CREATE TABLE t1 AS " - + "SELECT * FROM test4 JOIN test5 " - + "ON test4.id = test5.rowkey;"); - - // Then: did not throw. - } - @Test public void shouldGetSingleValueSchemaWrappingFromPropertiesBeforeConfig() { // Given: @@ -412,7 +352,7 @@ public void shouldGetSingleValueSchemaWrappingFromConfig() { execute(CREATE_TABLE_TEST4); // When: - execute("CREATE TABLE TEST5 AS SELECT rowkey, COL0 FROM TEST4;"); + execute("CREATE TABLE TEST5 AS SELECT ID, COL0 FROM TEST4;"); // Then: assertThat(engineMetastore.getSource(SourceName.of("TEST5")), @@ -426,7 +366,7 @@ public void shouldDefaultToWrappingSingleValueSchemas() { execute(CREATE_TABLE_TEST4); // When: - execute("CREATE TABLE TEST5 AS SELECT ROWKEY, COL0 FROM TEST4;"); + execute("CREATE TABLE TEST5 AS SELECT ID, COL0 FROM TEST4;"); // Then: assertThat(engineMetastore.getSource(SourceName.of("TEST5")), diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/LogicalPlannerTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/LogicalPlannerTest.java index fca2faa28785..4dc151dee633 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/LogicalPlannerTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/LogicalPlannerTest.java @@ -20,7 +20,6 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.is; import io.confluent.ksql.execution.expression.tree.ComparisonExpression; import io.confluent.ksql.execution.expression.tree.ComparisonExpression.Type; @@ -43,7 +42,6 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.MetaStoreFixture; import java.util.Collections; -import java.util.Optional; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -120,7 +118,6 @@ public void testSimpleLeftJoinFilterLogicalPlan() { assertThat(logicalPlan.getSources().get(0), instanceOf(ProjectNode.class)); final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - assertThat(projectNode.getKeyField().ref(), is(Optional.empty())); assertThat(projectNode.getSchema().value().size(), equalTo(5)); assertThat(projectNode.getSources().get(0), instanceOf(FilterNode.class)); @@ -156,24 +153,22 @@ public void shouldAddProjectionWithSourceAliasPrefixForJoinSources() { final JoinNode joinNode = (JoinNode) logicalPlan.getSources().get(0).getSources().get(0); final ProjectNode left = (ProjectNode) joinNode.getSources().get(0); assertThat(left.getSelectExpressions(), contains( - selectCol("COL0", "T1_COL0"), selectCol("COL1", "T1_COL1"), selectCol("COL2", "T1_COL2"), selectCol("COL3", "T1_COL3"), selectCol("COL4", "T1_COL4"), selectCol("COL5", "T1_COL5"), selectCol("ROWTIME", "T1_ROWTIME"), - selectCol("ROWKEY", "T1_ROWKEY") + selectCol("COL0", "T1_COL0") )); final ProjectNode right = (ProjectNode) joinNode.getSources().get(1); assertThat(right.getSelectExpressions(), contains( - selectCol("COL0", "T2_COL0"), selectCol("COL1", "T2_COL1"), selectCol("COL2", "T2_COL2"), selectCol("COL3", "T2_COL3"), selectCol("COL4", "T2_COL4"), selectCol("ROWTIME", "T2_ROWTIME"), - selectCol("ROWKEY", "T2_ROWKEY") + selectCol("COL0", "T2_COL0") )); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java index ff3bbb705c9e..30091e3f2ef7 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java @@ -28,7 +28,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.is; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; @@ -61,7 +60,6 @@ import java.util.IdentityHashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.kafka.common.serialization.Serde; @@ -150,7 +148,7 @@ public void shouldUseConsistentOrderInPreAggSelectMapper() { final ValueTransformerWithKey preAggSelectMapper = valueTransformers.get(1).get(); preAggSelectMapper.init(ctx); final GenericRow result = (GenericRow) preAggSelectMapper - .transform(null, genericRow(0L, "1", "2", 3.0D, "rowtime", "rowkey")); + .transform(null, genericRow("1", "2", 3.0D, null, null, "rowtime", 0L)); assertThat("should select col0, col1, col2, col3", result.values(), contains(0L, "1", "2", 3.0)); } @@ -305,26 +303,6 @@ public void shouldCreateLoggers() { )); } - @Test - public void shouldGroupByFunction() { - // Given: - final SchemaKStream stream = buildQuery("SELECT UCASE(col1), sum(col3), count(col3) FROM test1 " - + "GROUP BY UCASE(col1) EMIT CHANGES;"); - - // Then: - assertThat(stream.getKeyField().ref(), is(Optional.empty())); - } - - @Test - public void shouldGroupByArithmetic() { - // Given: - final SchemaKStream stream = buildQuery("SELECT col0 + 10, sum(col3), count(col3) FROM test1 " - + "GROUP BY col0 + 10 EMIT CHANGES;"); - - // Then: - assertThat(stream.getKeyField().ref(), is(Optional.empty())); - } - private SchemaKStream buildQuery(final String queryString) { return buildQuery(queryString, KSQL_CONFIG); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java index 8ff4568a6802..d75bbfb966ed 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java @@ -43,7 +43,6 @@ import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.metastore.model.KsqlTable; import io.confluent.ksql.model.WindowType; @@ -106,7 +105,6 @@ public class DataSourceNodeTest { .valueColumn(KEY, SqlTypes.STRING) .build(); - private static final KeyField KEY_FIELD = KeyField.of(FIELD1); private static final TimestampColumn TIMESTAMP_COLUMN = new TimestampColumn(TIMESTAMP_FIELD, Optional.empty()); @@ -115,7 +113,6 @@ public class DataSourceNodeTest { SOURCE_NAME, REAL_SCHEMA, SerdeOption.none(), - KEY_FIELD, Optional.of( new TimestampColumn( ColumnName.of("timestamp"), @@ -179,7 +176,7 @@ public void before() { when(dataSource.getKsqlTopic()).thenReturn(topic); when(dataSource.getDataSourceType()).thenReturn(DataSourceType.KTABLE); - when(schemaKStreamFactory.create(any(), any(), any(), any())) + when(schemaKStreamFactory.create(any(), any(), any())) .thenAnswer(inv -> inv.getArgument(1) .getDataSourceType() == DataSourceType.KSTREAM ? stream : table @@ -227,15 +224,6 @@ public void shouldBeOfTypeSchemaKStreamWhenDataSourceIsKsqlStream() { assertThat(realStream.getClass(), equalTo(SchemaKStream.class)); } - @Test - public void shouldBuildStreamWithSameKeyField() { - // When: - final SchemaKStream stream = buildStream(node); - - // Then: - assertThat(stream.getKeyField(), is(node.getKeyField())); - } - @Test public void shouldBuildSchemaKTableWhenKTableSource() { // Given: @@ -243,7 +231,6 @@ public void shouldBuildSchemaKTableWhenKTableSource() { SourceName.of("datasource"), REAL_SCHEMA, SerdeOption.none(), - KeyField.of(ColumnName.of("field1")), Optional.of(TIMESTAMP_COLUMN), false, new KsqlTopic( @@ -297,7 +284,7 @@ public void shouldBuildSourceStreamWithCorrectTimestampIndex() { node.buildStream(ksqlStreamBuilder); // Then: - verify(schemaKStreamFactory).create(any(), any(), any(), any()); + verify(schemaKStreamFactory).create(any(), any(), any()); } // should this even be possible? if you are using a timestamp extractor then shouldn't the name @@ -311,7 +298,7 @@ public void shouldBuildSourceStreamWithCorrectTimestampIndexForQualifiedFieldNam node.buildStream(ksqlStreamBuilder); // Then: - verify(schemaKStreamFactory).create(any(), any(), any(), any()); + verify(schemaKStreamFactory).create(any(), any(), any()); } @Test @@ -328,8 +315,7 @@ public void shouldBuildSourceStreamWithCorrectParams() { verify(schemaKStreamFactory).create( same(ksqlStreamBuilder), same(dataSource), - stackerCaptor.capture(), - same(node.getKeyField()) + stackerCaptor.capture() ); assertThat( stackerCaptor.getValue().getQueryContext().getContext(), @@ -349,8 +335,7 @@ public void shouldBuildSourceStreamWithCorrectParamsWhenBuildingTable() { verify(schemaKStreamFactory).create( same(ksqlStreamBuilder), same(dataSource), - stackerCaptor.capture(), - same(node.getKeyField()) + stackerCaptor.capture() ); assertThat( stackerCaptor.getValue().getQueryContext().getContext(), @@ -439,7 +424,6 @@ public void shouldThrowIfProjectionDoesNotContainKeyColumns() { private void givenNodeWithMockSource() { when(dataSource.getSchema()).thenReturn(REAL_SCHEMA); - when(dataSource.getKeyField()).thenReturn(KEY_FIELD); node = new DataSourceNode( PLAN_NODE_ID, dataSource, diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java index 8bd9b3cddaa3..7b2c562c4c62 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java @@ -46,7 +46,6 @@ import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.WithinExpression; @@ -137,8 +136,6 @@ public class JoinNodeTest { private static final ColumnName LEFT_JOIN_FIELD_REF = ColumnName.of("C0"); - private static final KeyField leftJoinField = KeyField.of(LEFT_JOIN_FIELD_REF); - private static final Optional WITHIN_EXPRESSION = Optional.of(new WithinExpression(10, TimeUnit.SECONDS)); @@ -205,8 +202,6 @@ public void setUp() { when(left.getPartitions(mockKafkaTopicClient)).thenReturn(2); when(right.getPartitions(mockKafkaTopicClient)).thenReturn(2); - when(left.getKeyField()).thenReturn(KeyField.of(LEFT_JOIN_FIELD_REF)); - when(left.getAlias()).thenReturn(LEFT_ALIAS); when(right.getAlias()).thenReturn(RIGHT_ALIAS); @@ -221,15 +216,6 @@ public void setUp() { setUpSource(right, OTHER_FORMAT, rightSource, "Foobar2"); } - @Test - public void shouldReturnLeftJoinKeyAsKeyField() { - // When: - final JoinNode joinNode = new JoinNode(nodeId, LEFT, joinKey, true, left, right, empty()); - - // Then: - assertThat(joinNode.getKeyField().ref(), is(Optional.of(LEFT_JOIN_FIELD_REF))); - } - @Test public void shouldBuildSourceNode() { setupTopicClientExpectations(1, 1); @@ -290,7 +276,6 @@ public void shouldPerformStreamToStreamLeftJoin() { verify(leftSchemaKStream).leftJoin( rightSchemaKStream, SYNTH_KEY, - leftJoinField, WITHIN_EXPRESSION.get().joinWindow(), VALUE_FORMAT, OTHER_FORMAT, @@ -314,7 +299,6 @@ public void shouldPerformStreamToStreamInnerJoin() { verify(leftSchemaKStream).join( rightSchemaKStream, SYNTH_KEY, - leftJoinField, WITHIN_EXPRESSION.get().joinWindow(), VALUE_FORMAT, OTHER_FORMAT, @@ -338,7 +322,6 @@ public void shouldPerformStreamToStreamOuterJoin() { verify(leftSchemaKStream).outerJoin( rightSchemaKStream, SYNTH_KEY, - KeyField.none(), WITHIN_EXPRESSION.get().joinWindow(), VALUE_FORMAT, OTHER_FORMAT, @@ -403,7 +386,6 @@ public void shouldHandleJoinIfTableHasNoKeyAndJoinFieldIsRowKey() { verify(leftSchemaKStream).leftJoin( rightSchemaKTable, SYNTH_KEY, - leftJoinField, VALUE_FORMAT, CONTEXT_STACKER ); @@ -424,7 +406,6 @@ public void shouldPerformStreamToTableLeftJoin() { verify(leftSchemaKStream).leftJoin( rightSchemaKTable, SYNTH_KEY, - leftJoinField, VALUE_FORMAT, CONTEXT_STACKER ); @@ -445,7 +426,6 @@ public void shouldPerformStreamToTableInnerJoin() { verify(leftSchemaKStream).join( rightSchemaKTable, SYNTH_KEY, - leftJoinField, VALUE_FORMAT, CONTEXT_STACKER ); @@ -507,7 +487,6 @@ public void shouldPerformTableToTableInnerJoin() { verify(leftSchemaKTable).join( rightSchemaKTable, SYNTH_KEY, - leftJoinField, CONTEXT_STACKER ); } @@ -527,7 +506,6 @@ public void shouldPerformTableToTableLeftJoin() { verify(leftSchemaKTable).leftJoin( rightSchemaKTable, SYNTH_KEY, - leftJoinField, CONTEXT_STACKER ); } @@ -547,7 +525,6 @@ public void shouldPerformTableToTableOuterJoin() { verify(leftSchemaKTable).outerJoin( rightSchemaKTable, SYNTH_KEY, - KeyField.none(), CONTEXT_STACKER ); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java index 70b8c8c7337c..5743be3659f4 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java @@ -34,7 +34,6 @@ import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.query.QueryId; @@ -74,9 +73,6 @@ public class KsqlStructuredDataOutputNodeTest { .valueColumn(ColumnName.of("key"), SqlTypes.STRING) .build(); - private static final KeyField KEY_FIELD = KeyField - .of(ColumnName.of("key")); - private static final PlanNodeId PLAN_NODE_ID = new PlanNodeId("0"); private static final ValueFormat JSON_FORMAT = ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())); @@ -239,7 +235,6 @@ private void buildNode() { sourceNode, SCHEMA, Optional.empty(), - KEY_FIELD, ksqlTopic, OptionalInt.empty(), createInto, diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/PlanNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/PlanNodeTest.java index db8c97716f03..054d3827da07 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/PlanNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/PlanNodeTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -122,11 +121,6 @@ protected TestPlanNode( super(id, nodeOutputType, schema, sourceName); } - @Override - public KeyField getKeyField() { - return null; - } - @Override public List getSources() { return ImmutableList.of(source1, source2); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/RepartitionNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/RepartitionNodeTest.java index b056191bea3b..204cfc9d5428 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/RepartitionNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/RepartitionNodeTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.Projection; @@ -88,7 +87,7 @@ public void setUp() { when(parent.resolveSelectStar(any())).thenReturn(PARENT_COL_NAMES.stream()); repartitionNode = new RepartitionNode(PLAN_ID, parent, SCHEMA, originalPartitionBy, - rewrittenPartitionBy, KeyField.none(), false); + rewrittenPartitionBy, false); } @Test @@ -172,11 +171,11 @@ public void shouldNotThrowIfProjectionHasPartitionBy() { private void givenAnyKeyEnabled() { repartitionNode = new RepartitionNode(PLAN_ID, parent, SCHEMA, originalPartitionBy, - rewrittenPartitionBy, KeyField.none(), false); + rewrittenPartitionBy, false); } private void givenInternalRepartition() { repartitionNode = new RepartitionNode(PLAN_ID, parent, SCHEMA, originalPartitionBy, - rewrittenPartitionBy, KeyField.none(), true); + rewrittenPartitionBy, true); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java index 2d51517dd186..0a60f62fec2a 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java @@ -38,7 +38,6 @@ import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.MutableMetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; @@ -133,7 +132,6 @@ public void setUp() { SourceName.of("SOURCE"), SCHEMA, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, sourceTopic diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/security/KsqlAuthorizationValidatorImplTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/security/KsqlAuthorizationValidatorImplTest.java index b14b201dfd9a..4b8a6cb7d42c 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/security/KsqlAuthorizationValidatorImplTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/security/KsqlAuthorizationValidatorImplTest.java @@ -27,7 +27,6 @@ import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.MutableMetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; @@ -387,7 +386,6 @@ private void givenStreamWithTopic( SourceName.of(streamName.toUpperCase()), SCHEMA, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, sourceTopic diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedStreamTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedStreamTest.java index 352665b849c2..7a47939ebf2c 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedStreamTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedStreamTest.java @@ -31,7 +31,6 @@ import io.confluent.ksql.execution.windows.WindowTimeClause; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.InternalFunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.model.WindowType; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.FunctionName; @@ -84,8 +83,6 @@ public class SchemaKGroupedStreamTest { ColumnName.of("IN0") ); - @Mock - private KeyField keyField; @Mock private KsqlConfig config; @Mock @@ -115,7 +112,6 @@ public void setUp() { sourceStep, IN_SCHEMA, keyFormat, - keyField, config, functionRegistry ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java index a7261348692e..4e261077a83a 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java @@ -29,7 +29,6 @@ import io.confluent.ksql.execution.plan.ExecutionStep; import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.function.InternalFunctionRegistry; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.parser.tree.WindowExpression; @@ -132,7 +131,6 @@ private SchemaKGroupedTable buildSchemaKGroupedTable() { mock(ExecutionStep.class), IN_SCHEMA, keyFormat, - KeyField.of(IN_SCHEMA.value().get(0).name()), ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKSourceFactoryTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKSourceFactoryTest.java index aed1ca7eb28b..595c8871fb3d 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKSourceFactoryTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKSourceFactoryTest.java @@ -36,7 +36,6 @@ import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.SystemColumns; @@ -58,8 +57,6 @@ @RunWith(MockitoJUnitRunner.class) public class SchemaKSourceFactoryTest { - private static final KeyField KEY_FIELD = KeyField.none(); - private static final LogicalSchema SCHEMA = LogicalSchema.builder() .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) .valueColumn(ColumnName.of("FOO"), SqlTypes.INTEGER) @@ -125,8 +122,7 @@ public void shouldBuildWindowedStream() { final SchemaKStream result = SchemaKSourceFactory.buildSource( builder, dataSource, - contextStacker, - KEY_FIELD + contextStacker ); // Then: @@ -148,8 +144,7 @@ public void shouldBuildNonWindowedStream() { final SchemaKStream result = SchemaKSourceFactory.buildSource( builder, dataSource, - contextStacker, - KEY_FIELD + contextStacker ); // Then: @@ -171,8 +166,7 @@ public void shouldBuildWindowedTable() { final SchemaKStream result = SchemaKSourceFactory.buildSource( builder, dataSource, - contextStacker, - KEY_FIELD + contextStacker ); // Then: @@ -194,8 +188,7 @@ public void shouldBuildNonWindowedTable() { final SchemaKStream result = SchemaKSourceFactory.buildSource( builder, dataSource, - contextStacker, - KEY_FIELD + contextStacker ); // Then: diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java index 23a022ad6e55..eff8c7f3ad22 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java @@ -26,7 +26,6 @@ import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.expression.tree.ComparisonExpression; import io.confluent.ksql.execution.expression.tree.Expression; -import io.confluent.ksql.execution.expression.tree.FunctionCall; import io.confluent.ksql.execution.expression.tree.LongLiteral; import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.execution.plan.ExecutionStep; @@ -37,12 +36,9 @@ import io.confluent.ksql.execution.streams.StepSchemaResolver; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.metastore.model.KsqlTable; -import io.confluent.ksql.metastore.model.MetaStoreMatchers.KeyFieldMatchers; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.plan.FilterNode; import io.confluent.ksql.planner.plan.PlanNode; @@ -64,7 +60,6 @@ import io.confluent.ksql.util.Pair; import java.util.Collections; import java.util.List; -import java.util.Optional; import org.apache.kafka.streams.kstream.JoinWindows; import org.junit.Before; import org.junit.Test; @@ -84,8 +79,6 @@ public class SchemaKStreamTest { private final KsqlConfig ksqlConfig = new KsqlConfig(Collections.emptyMap()); private final MetaStore metaStore = MetaStoreFixture .getNewMetaStore(new InternalFunctionRegistry()); - private final KeyField validJoinKeyField = KeyField - .of(Optional.of(ColumnName.of("COL0"))); private final KeyFormat keyFormat = KeyFormat .nonWindowed(FormatInfo.of(FormatFactory.KAFKA.name())); private final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())); @@ -117,7 +110,6 @@ public void init() { tableSourceStep, ksqlTable.getSchema(), keyFormat, - ksqlTable.getKeyField(), ksqlConfig, functionRegistry); } @@ -214,7 +206,7 @@ public void shouldNotRepartitionIfSameKeyField() { public void shouldNotRepartitionIfRowkey() { // Given: final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT col0, col2, col3 FROM test1 PARTITION BY ROWKEY EMIT CHANGES;"); + "SELECT col0, col2, col3 FROM test1 PARTITION BY col0 EMIT CHANGES;"); final RepartitionNode repartitionNode = (RepartitionNode) logicalPlan.getSources().get(0).getSources().get(0); // When: @@ -225,52 +217,6 @@ public void shouldNotRepartitionIfRowkey() { assertThat(result, is(initialSchemaKStream)); } - @Test - public void shouldUpdateKeyOnPartitionByColumn() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT col0, col2, col3 FROM test1 PARTITION BY col2 EMIT CHANGES;"); - final RepartitionNode repartitionNode = (RepartitionNode) logicalPlan.getSources().get(0).getSources().get(0); - - // When: - final SchemaKStream result = initialSchemaKStream - .selectKey(repartitionNode.getPartitionBy(), childContextStacker); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("COL2")))); - } - - @Test - public void shouldUpdateKeyToNoneOnPartitionByMetaColumn() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT col0, col2, col3 FROM test1 PARTITION BY ROWTIME EMIT CHANGES;"); - final RepartitionNode repartitionNode = (RepartitionNode) logicalPlan.getSources().get(0).getSources().get(0); - - // When: - final SchemaKStream result = initialSchemaKStream - .selectKey(repartitionNode.getPartitionBy(), childContextStacker); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - - @Test - public void shouldUpdateKeyToNoneOnPartitionByExpression() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT col0, col2, col3 FROM test1 PARTITION BY col2 + 'foo' EMIT CHANGES;"); - final RepartitionNode repartitionNode = (RepartitionNode) logicalPlan.getSources().get(0).getSources().get(0); - - // When: - final SchemaKStream result = initialSchemaKStream - .selectKey(repartitionNode.getPartitionBy(), childContextStacker); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - @Test(expected = KsqlException.class) public void shouldThrowOnRepartitionByMissingField() { // Given: @@ -284,118 +230,6 @@ public void shouldThrowOnRepartitionByMissingField() { ); } - @Test - public void shouldUpdateKeyIfRenamed() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT col0 as NEWKEY, col2, col3 FROM test1 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldUpdateKeyIfRenamedViaFullyQualifiedName() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT test1.col0 as NEWKEY, col2, col3 FROM test1 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldUpdateKeyIfRenamedAndSourceIsAliased() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf( - "SELECT t.col0 as NEWKEY, col2, col3 FROM test1 t EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldPreserveKeyOnSelectStar() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf("SELECT * FROM test1 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), KeyFieldMatchers.hasName("COL0")); - } - - @Test - public void shouldUpdateKeyIfMovedToDifferentIndex() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf("SELECT col2, col0, col3 FROM test1 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - equalTo(KeyField.of(ColumnName.of("COL0")))); - } - - @Test - public void shouldDropKeyIfNotSelected() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf("SELECT col2, col3 FROM test1 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - - @Test - public void shouldHandleSourceWithoutKey() { - // Given: - final PlanNode logicalPlan = givenInitialKStreamOf("SELECT * FROM test4 EMIT CHANGES;"); - final ProjectNode projectNode = (ProjectNode) logicalPlan.getSources().get(0); - final List selectExpressions = projectNode.getSelectExpressions(); - - // When: - final SchemaKStream result = initialSchemaKStream - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - @Test(expected = UnsupportedOperationException.class) public void shouldFailRepartitionTable() { // Given: @@ -484,23 +318,6 @@ public void shouldBuildStepForFilter() { ); } - @Test - public void shouldSelectKey() { - // Given: - givenInitialKStreamOf("SELECT col0, col2, col3 FROM test1 WHERE col0 > 100 EMIT CHANGES;"); - - final KeyField expected = KeyField - .of(ColumnName.of("COL1")); - - // When: - final SchemaKStream rekeyedSchemaKStream = initialSchemaKStream.selectKey( - new UnqualifiedColumnReferenceExp(ColumnName.of("COL1")), - childContextStacker); - - // Then: - assertThat(rekeyedSchemaKStream.getKeyField(), is(expected)); - } - @Test public void shouldBuildStepForSelectKey() { // Given: @@ -542,26 +359,6 @@ public void shouldBuildSchemaForSelectKey() { ); } - @Test - public void testGroupByKey() { - // Given: - givenInitialKStreamOf("SELECT col0, col1 FROM test1 WHERE col0 > 100 EMIT CHANGES;"); - - final List groupBy = Collections.singletonList( - new UnqualifiedColumnReferenceExp(ColumnName.of("COL0")) - ); - - // When: - final SchemaKGroupedStream groupedSchemaKStream = initialSchemaKStream.groupBy( - valueFormat, - groupBy, - childContextStacker - ); - - // Then: - assertThat(groupedSchemaKStream.getKeyField().ref(), is(Optional.of(ColumnName.of("COL0")))); - } - @Test public void shouldBuildStepForGroupByKey() { // Given: @@ -667,50 +464,10 @@ public void shouldBuildSchemaForGroupBy() { ); } - @Test - public void testGroupByMultipleColumns() { - // Given: - givenInitialKStreamOf("SELECT col0, col1 FROM test1 WHERE col0 > 100 EMIT CHANGES;"); - - final List groupBy = ImmutableList.of( - new UnqualifiedColumnReferenceExp(ColumnName.of("COL1")), - new UnqualifiedColumnReferenceExp(ColumnName.of("COL0")) - ); - - // When: - final SchemaKGroupedStream groupedSchemaKStream = initialSchemaKStream.groupBy( - valueFormat, - groupBy, - childContextStacker - ); - - // Then: - assertThat(groupedSchemaKStream.getKeyField().ref(), is(Optional.empty())); - } - - @Test - public void testGroupByMoreComplexExpression() { - // Given: - givenInitialKStreamOf("SELECT col0, col1 FROM test1 WHERE col0 > 100 EMIT CHANGES;"); - - final Expression groupBy = new FunctionCall(FunctionName.of("UCASE"), ImmutableList.of(COL1)); - - // When: - final SchemaKGroupedStream groupedSchemaKStream = initialSchemaKStream.groupBy( - valueFormat, - ImmutableList.of(groupBy), - childContextStacker - ); - - // Then: - assertThat(groupedSchemaKStream.getKeyField().ref(), is(Optional.empty())); - } - @FunctionalInterface private interface StreamStreamJoin { SchemaKStream join( SchemaKStream otherSchemaKStream, - KeyField keyField, JoinWindows joinWindows, ValueFormat leftFormat, ValueFormat rightFormat, @@ -723,7 +480,6 @@ private interface StreamTableJoin { SchemaKStream join( SchemaKTable other, ColumnName keyNameCol, - KeyField keyField, ValueFormat leftFormat, QueryContext.Stacker contextStacker ); @@ -743,7 +499,6 @@ public void shouldBuildStepForStreamTableJoin() { final SchemaKStream joinedKStream = testcase.right.join( schemaKTable, KEY, - validJoinKeyField, valueFormat, childContextStacker ); @@ -780,7 +535,6 @@ public void shouldBuildSchemaForStreamTableJoin() { final SchemaKStream joinedKStream = testcase.right.join( schemaKTable, KEY, - validJoinKeyField, valueFormat, childContextStacker ); @@ -796,13 +550,12 @@ public void shouldBuildSchemaForStreamTableJoin() { private SchemaKStream buildSchemaKStream( final LogicalSchema schema, - final KeyField keyField, - final ExecutionStep sourceStep) { + final ExecutionStep sourceStep + ) { return new SchemaKStream( sourceStep, schema, keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -820,7 +573,6 @@ private LogicalSchema buildJoinSchema(final KsqlStream stream) { private SchemaKStream buildSchemaKStreamForJoin(final KsqlStream ksqlStream) { return buildSchemaKStream( buildJoinSchema(ksqlStream), - KeyField.none(), sourceStep ); } @@ -835,7 +587,6 @@ private PlanNode givenInitialKStreamOf(final String selectQuery) { sourceStep, logicalPlan.getTheSourceNode().getSchema(), keyFormat, - logicalPlan.getTheSourceNode().getKeyField(), ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java index a43489bf9b23..a5cae6bca753 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java @@ -15,10 +15,9 @@ package io.confluent.ksql.structured; -import static io.confluent.ksql.GenericRow.genericRow; +import static io.confluent.ksql.schema.ksql.ColumnMatchers.keyColumn; import static io.confluent.ksql.schema.ksql.ColumnMatchers.valueColumn; import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.mock; @@ -63,14 +62,11 @@ import io.confluent.ksql.execution.streams.StreamJoinedFactory; import io.confluent.ksql.execution.streams.StreamsFactories; import io.confluent.ksql.execution.streams.StreamsUtil; -import io.confluent.ksql.execution.util.StructKeyUtil; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.MetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlTable; -import io.confluent.ksql.metastore.model.MetaStoreMatchers.KeyFieldMatchers; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.planner.plan.FilterNode; @@ -94,20 +90,16 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Grouped; import org.apache.kafka.streams.kstream.KGroupedTable; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; -import org.easymock.Capture; import org.easymock.EasyMock; import org.hamcrest.Matchers; import org.junit.Before; @@ -137,7 +129,6 @@ public class SchemaKTableTest { private KsqlTable secondKsqlTable; private InternalFunctionRegistry functionRegistry; private KTable mockKTable; - private KeyField validKeyField; private SchemaKTable firstSchemaKTable; private SchemaKTable secondSchemaKTable; private StepSchemaResolver schemaResolver; @@ -185,8 +176,6 @@ public void init() { )); mockKTable = EasyMock.niceMock(KTable.class); - validKeyField = KeyField - .of(Optional.of(ColumnName.of("TEST2_COL0"))); firstSchemaKTable = buildSchemaKTableForJoin(ksqlTable, mockKTable); secondSchemaKTable = buildSchemaKTableForJoin(secondKsqlTable, secondKTable); @@ -217,13 +206,11 @@ private ExecutionStep buildSourceStep(final LogicalSchema schema, final KTable k private SchemaKTable buildSchemaKTable( final LogicalSchema schema, - final KeyField keyField, final KTable kTable) { return new SchemaKTable<>( buildSourceStep(schema, kTable), schema, keyFormat, - keyField, ksqlConfig, functionRegistry ); @@ -234,7 +221,6 @@ private SchemaKTable buildSchemaKTableFromPlan(final PlanNode logicalPlan) { buildSourceStep(logicalPlan.getTheSourceNode().getSchema(), kTable), logicalPlan.getTheSourceNode().getSchema(), keyFormat, - logicalPlan.getTheSourceNode().getKeyField(), ksqlConfig, functionRegistry ); @@ -243,13 +229,8 @@ private SchemaKTable buildSchemaKTableFromPlan(final PlanNode logicalPlan) { private SchemaKTable buildSchemaKTable(final KsqlTable ksqlTable, final KTable kTable) { final LogicalSchema schema = ksqlTable.getSchema(); - final Optional newKeyName = ksqlTable.getKeyField().ref(); - - final KeyField keyFieldWithAlias = KeyField.of(newKeyName); - return buildSchemaKTable( schema, - keyFieldWithAlias, kTable ); } @@ -266,7 +247,6 @@ private LogicalSchema buildJoinSchema(final KsqlTable table) { private SchemaKTable buildSchemaKTableForJoin(final KsqlTable ksqlTable, final KTable kTable) { return buildSchemaKTable( buildJoinSchema(ksqlTable), - KeyField.none(), kTable ); } @@ -393,16 +373,14 @@ public void shouldBuildSchemaKTableWithCorrectSchemaForFilter() { ); // Then: - final SourceName test2 = SourceName.of("TEST2"); - assertThat(filteredSchemaKStream.getSchema().value(), contains( - valueColumn(test2, ColumnName.of("COL0"), SqlTypes.BIGINT), - valueColumn(test2, ColumnName.of("COL1"), SqlTypes.STRING), - valueColumn(test2, ColumnName.of("COL2"), SqlTypes.STRING), - valueColumn(test2, ColumnName.of("COL3"), SqlTypes.DOUBLE), - valueColumn(test2, ColumnName.of("COL4"), SqlTypes.BOOLEAN), - valueColumn(test2, ColumnName.of("ROWTIME"), SqlTypes.BIGINT), - - valueColumn(test2, ColumnName.of("ROWKEY"), SqlTypes.BIGINT) + assertThat(filteredSchemaKStream.getSchema().columns(), contains( + keyColumn(ColumnName.of("COL0"), SqlTypes.BIGINT), + valueColumn(ColumnName.of("COL1"), SqlTypes.STRING), + valueColumn(ColumnName.of("COL2"), SqlTypes.STRING), + valueColumn(ColumnName.of("COL3"), SqlTypes.DOUBLE), + valueColumn(ColumnName.of("COL4"), SqlTypes.BOOLEAN), + valueColumn(ColumnName.of("ROWTIME"), SqlTypes.BIGINT), + valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT) )); } @@ -479,7 +457,6 @@ public void testGroupBy() { // Then: assertThat(groupedSchemaKTable, instanceOf(SchemaKGroupedTable.class)); - assertThat(groupedSchemaKTable.getKeyField().ref(), is(Optional.empty())); } @Test @@ -562,51 +539,6 @@ public void shouldUseOpNameForGrouped() { verify(mockKTable, groupedFactory); } - @SuppressWarnings("unchecked") - @Test - public void shouldGroupKeysCorrectly() { - // set up a mock KTable and KGroupedTable for the test. Set up the KTable to - // capture the mapper that is passed in to produce new keys - final KTable mockKTable = mock(KTable.class); - final KGroupedTable mockKGroupedTable = mock(KGroupedTable.class); - final Capture capturedKeySelector = Capture.newInstance(); - expect(mockKTable.filter(anyObject(Predicate.class))).andReturn(mockKTable); - expect(mockKTable.groupBy(capture(capturedKeySelector), anyObject(Grouped.class))) - .andReturn(mockKGroupedTable); - replay(mockKTable, mockKGroupedTable); - - // Build our test object from the mocks - final String selectQuery = "SELECT col0, col1, col2 FROM test2 EMIT CHANGES;"; - final PlanNode logicalPlan = buildLogicalPlan(selectQuery); - initialSchemaKTable = new SchemaKTable( - buildSourceStep(logicalPlan.getTheSourceNode().getSchema(), mockKTable), - logicalPlan.getTheSourceNode().getSchema(), - keyFormat, - logicalPlan.getTheSourceNode().getKeyField(), - ksqlConfig, - functionRegistry - ); - - final List groupByExpressions = Arrays.asList(TEST_2_COL_2, TEST_2_COL_1); - - // Call groupBy and extract the captured mapper - final SchemaKGroupedTable result = initialSchemaKTable.groupBy( - valueFormat, groupByExpressions, childContextStacker); - result.getSourceTableStep().build(planBuilder); - verify(mockKTable, mockKGroupedTable); - final KeyValueMapper keySelector = capturedKeySelector.getValue(); - final GenericRow value = genericRow(100, "foo", "bar", 0, "key"); - final KeyValue keyValue = - (KeyValue) keySelector.apply("key", value); - - // Validate that the captured mapper produces the correct key - assertThat(keyValue.key, equalTo(StructKeyUtil.keyBuilder( - ColumnName.of("KSQL_COL_0"), - SqlTypes.STRING - ).build("bar|+|foo"))); - assertThat(keyValue.value, equalTo(value)); - } - @SuppressWarnings("unchecked") @Test public void shouldPerformTableToTableLeftJoin() { @@ -617,12 +549,11 @@ public void shouldPerformTableToTableLeftJoin() { replay(mockKTable); final SchemaKStream joinedKStream = firstSchemaKTable - .leftJoin(secondSchemaKTable, KEY, validKeyField, childContextStacker); + .leftJoin(secondSchemaKTable, KEY, childContextStacker); ((SchemaKTable) joinedKStream).getSourceTableStep().build(planBuilder); verify(mockKTable); assertThat(joinedKStream, instanceOf(SchemaKTable.class)); - assertThat(joinedKStream.getKeyField(), is(validKeyField)); } @SuppressWarnings("unchecked") @@ -635,12 +566,11 @@ public void shouldPerformTableToTableInnerJoin() { replay(mockKTable); final SchemaKStream joinedKStream = firstSchemaKTable - .join(secondSchemaKTable, KEY, validKeyField, childContextStacker); + .join(secondSchemaKTable, KEY, childContextStacker); ((SchemaKTable) joinedKStream).getSourceTableStep().build(planBuilder); verify(mockKTable); assertThat(joinedKStream, instanceOf(SchemaKTable.class)); - assertThat(joinedKStream.getKeyField(), is(validKeyField)); } @SuppressWarnings("unchecked") @@ -653,12 +583,11 @@ public void shouldPerformTableToTableOuterJoin() { replay(mockKTable); final SchemaKStream joinedKStream = firstSchemaKTable - .outerJoin(secondSchemaKTable, ColumnName.of("KEY"), validKeyField, childContextStacker); + .outerJoin(secondSchemaKTable, ColumnName.of("KEY"), childContextStacker); ((SchemaKTable) joinedKStream).getSourceTableStep().build(planBuilder); verify(mockKTable); assertThat(joinedKStream, instanceOf(SchemaKTable.class)); - assertThat(joinedKStream.getKeyField(), is(validKeyField)); } @FunctionalInterface @@ -667,7 +596,6 @@ private interface Join { SchemaKTable join( SchemaKTable schemaKTable, ColumnName keyColName, - KeyField keyField, QueryContext.Stacker contextStacker ); } @@ -685,7 +613,7 @@ public void shouldBuildStepForTableTableJoin() { for (final Pair testCase : cases) { // When: final SchemaKTable result = testCase.right - .join(secondSchemaKTable, KEY, validKeyField, childContextStacker); + .join(secondSchemaKTable, KEY, childContextStacker); // Then: assertThat( @@ -716,7 +644,7 @@ public void shouldBuildSchemaForTableTableJoin() { for (final Pair testCase : cases) { // When: final SchemaKTable result = testCase.right - .join(secondSchemaKTable, KEY, validKeyField, childContextStacker); + .join(secondSchemaKTable, KEY, childContextStacker); // Then: assertThat(result.getSchema(), is(schemaResolver.resolve( @@ -725,129 +653,6 @@ public void shouldBuildSchemaForTableTableJoin() { } } - @Test - public void shouldUpdateKeyIfRenamed() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT col0 as NEWKEY, col2, col3 FROM test1 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldUpdateKeyIfRenamedViaFullyQualifiedName() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT test1.col0 as NEWKEY, col2, col3 FROM test1 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldUpdateKeyIfRenamedAndSourceIsAliased() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT t.col0 as NEWKEY, col2, col3 FROM test1 t EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("NEWKEY")))); - } - - @Test - public void shouldPreserveKeyOnSelectStar() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT * FROM test1 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), KeyFieldMatchers.hasName("COL0")); - } - - @Test - public void shouldUpdateKeyIfMovedToDifferentIndex() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT col2, col0, col3 FROM test1 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("COL0")))); - } - - @Test - public void shouldDropKeyIfNotSelected() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT col2, col3 FROM test1 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - - @Test - public void shouldHandleSourceWithoutKey() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT * FROM test4 EMIT CHANGES;"); - - // When: - final SchemaKTable result = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - // Then: - assertThat(result.getKeyField(), is(KeyField.none())); - } - - @Test - public void shouldSetKeyOnGroupBySingleExpressionThatIsInProjection() { - // Given: - final List selectExpressions = givenInitialKTableOf( - "SELECT * FROM test2 EMIT CHANGES;"); - - final SchemaKTable selected = initialSchemaKTable - .select(selectExpressions, childContextStacker, queryBuilder); - - final List groupByExprs = ImmutableList.of( - new UnqualifiedColumnReferenceExp(ColumnName.of("COL0")) - ); - - // When: - final SchemaKGroupedTable result = selected - .groupBy(valueFormat, groupByExprs, childContextStacker); - - // Then: - assertThat(result.getKeyField(), - is(KeyField.of(ColumnName.of("COL0")))); - } - private List givenInitialKTableOf(final String selectQuery) { final PlanNode logicalPlan = AnalysisTestUtil.buildLogicalPlan( ksqlConfig, @@ -859,7 +664,6 @@ private List givenInitialKTableOf(final String selectQuery) { buildSourceStep(logicalPlan.getTheSourceNode().getSchema(), kTable), logicalPlan.getTheSourceNode().getSchema(), keyFormat, - logicalPlan.getTheSourceNode().getKeyField(), ksqlConfig, functionRegistry ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/SourceTopicsExtractorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/SourceTopicsExtractorTest.java index 26583ee8b865..4f4bd9123bbc 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/SourceTopicsExtractorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/SourceTopicsExtractorTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.MutableMetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; @@ -170,7 +169,6 @@ private void givenStreamWithTopic( SourceName.of(streamName.toUpperCase()), SCHEMA, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, sourceTopic diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java index 309d4692680a..3dc720b470eb 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java @@ -32,7 +32,6 @@ import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.MutableMetaStore; -import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; @@ -106,12 +105,11 @@ public void setUp() { ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())) ); - final KsqlStream source = new KsqlStream<>( + final KsqlStream source = new KsqlStream<>( "", SourceName.of("SOURCE"), SCHEMA, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, sourceTopic @@ -124,12 +122,11 @@ public void setUp() { ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())) ); - final KsqlStream joinSource = new KsqlStream<>( + final KsqlStream joinSource = new KsqlStream<>( "", SourceName.of("J_SOURCE"), SCHEMA, SerdeOption.none(), - KeyField.none(), Optional.empty(), false, joinTopic diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java index 5d6158e6279b..56cfb5d604d8 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java @@ -17,39 +17,38 @@ import static io.confluent.ksql.GenericRow.genericRow; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.Multimap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOption; -import java.util.Map; public class ItemDataProvider extends TestDataProvider { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("ID"), SqlTypes.STRING) + .keyColumn(ColumnName.of("ID"), SqlTypes.STRING) .valueColumn(ColumnName.of("DESCRIPTION"), SqlTypes.STRING) .build(); private static final PhysicalSchema PHYSICAL_SCHEMA = PhysicalSchema .from(LOGICAL_SCHEMA, SerdeOption.none()); - private static final Map ROWS = ImmutableMap.builder() - .put("ITEM_1", genericRow("ITEM_1", "home cinema")) - .put("ITEM_2", genericRow("ITEM_2", "clock radio")) - .put("ITEM_3", genericRow("ITEM_3", "road bike")) - .put("ITEM_4", genericRow("ITEM_4", "mountain bike")) - .put("ITEM_5", genericRow("ITEM_5", "snowboard")) - .put("ITEM_6", genericRow("ITEM_6", "iphone 10")) - .put("ITEM_7", genericRow("ITEM_7", "gopro")) - .put("ITEM_8", genericRow("ITEM_8", "cat")) + private static final Multimap ROWS = ImmutableListMultimap + .builder() + .put("ITEM_1", genericRow("home cinema")) + .put("ITEM_2", genericRow("clock radio")) + .put("ITEM_3", genericRow("road bike")) + .put("ITEM_4", genericRow("mountain bike")) + .put("ITEM_5", genericRow("snowboard")) + .put("ITEM_6", genericRow("iphone 10")) + .put("ITEM_7", genericRow("gopro")) + .put("ITEM_8", genericRow("cat")) .build(); public ItemDataProvider() { - super("ITEM", "ID", PHYSICAL_SCHEMA, ROWS); + super("ITEM", PHYSICAL_SCHEMA, ROWS); } } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/OrderDataProvider.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/OrderDataProvider.java index 0622cacfed6b..273b64635d53 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/OrderDataProvider.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/OrderDataProvider.java @@ -18,22 +18,22 @@ import static io.confluent.ksql.GenericRow.genericRow; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Multimap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOption; import java.util.Map; -public class OrderDataProvider extends TestDataProvider { +public class OrderDataProvider extends TestDataProvider { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("ORDERID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of("ORDERID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) .valueColumn(ColumnName.of("ORDERUNITS"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("TIMESTAMP"), SqlTypes.STRING) @@ -50,12 +50,12 @@ public class OrderDataProvider extends TestDataProvider { "key3", 3.0 ); - private static final Map ROWS = ImmutableMap.builder() + private static final Multimap ROWS = ImmutableListMultimap + .builder() .put( - 1L, + "ORDER_1", genericRow( 1L, - "ORDER_1", "ITEM_1", 10.0, "2018-01-01", @@ -63,10 +63,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 2L, + "ORDER_2", genericRow( 2L, - "ORDER_2", "ITEM_2", 20.0, "2018-01-02", @@ -74,10 +73,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 3L, + "ORDER_3", genericRow( 3L, - "ORDER_3", "ITEM_3", 30.0, "2018-01-03", @@ -85,10 +83,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 4L, + "ORDER_4", genericRow( 4L, - "ORDER_4", "ITEM_4", 40.0, "2018-01-04", @@ -96,10 +93,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 5L, + "ORDER_5", genericRow( 5L, - "ORDER_5", "ITEM_5", 50.0, "2018-01-05", @@ -107,10 +103,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 6L, + "ORDER_6", genericRow( 6L, - "ORDER_6", "ITEM_6", 60.0, "2018-01-06", @@ -118,10 +113,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 7L, + "ORDER_6", genericRow( 7L, - "ORDER_6", "ITEM_7", 70.0, "2018-01-07", @@ -129,10 +123,9 @@ public class OrderDataProvider extends TestDataProvider { MAP_FIELD )) .put( - 8L, + "ORDER_6", genericRow( 8L, - "ORDER_6", "ITEM_8", 80.0, "2018-01-08", @@ -142,6 +135,6 @@ public class OrderDataProvider extends TestDataProvider { .build(); public OrderDataProvider() { - super("ORDER", "ORDERTIME", PHYSICAL_SCHEMA, ROWS); + super("ORDER", PHYSICAL_SCHEMA, ROWS); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/PageViewDataProvider.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/PageViewDataProvider.java index 713099373544..a2a703541c32 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/PageViewDataProvider.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/PageViewDataProvider.java @@ -16,40 +16,39 @@ import static io.confluent.ksql.GenericRow.genericRow; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.Multimap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOption; -import java.util.Map; -public class PageViewDataProvider extends TestDataProvider { +public class PageViewDataProvider extends TestDataProvider { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.BIGINT) - .valueColumn(ColumnName.of("VIEWTIME"), SqlTypes.BIGINT) + .keyColumn(ColumnName.of("PAGEID"), SqlTypes.STRING) .valueColumn(ColumnName.of("USERID"), SqlTypes.STRING) - .valueColumn(ColumnName.of("PAGEID"), SqlTypes.STRING) + .valueColumn(ColumnName.of("VIEWTIME"), SqlTypes.BIGINT) .build(); private static final PhysicalSchema PHYSICAL_SCHEMA = PhysicalSchema .from(LOGICAL_SCHEMA, SerdeOption.none()); - private static final Map ROWS = ImmutableMap.builder() - .put(1L, genericRow(1L, "USER_1", "PAGE_1")) - .put(2L, genericRow(2L, "USER_2", "PAGE_2")) - .put(3L, genericRow(3L, "USER_4", "PAGE_3")) - .put(4L, genericRow(4L, "USER_3", "PAGE_4")) - .put(5L, genericRow(5L, "USER_0", "PAGE_5")) + private static final Multimap ROWS = ImmutableListMultimap + .builder() + .put("PAGE_1", genericRow("USER_1", 1L)) + .put("PAGE_2", genericRow("USER_2", 2L)) + .put("PAGE_3", genericRow("USER_4", 3L)) + .put("PAGE_4", genericRow("USER_3", 4L)) + .put("PAGE_5", genericRow("USER_0", 5L)) // Duplicate page views from different users. - .put(6L, genericRow(6L, "USER_2", "PAGE_5")) - .put(7L, genericRow(7L, "USER_3", "PAGE_5")) + .put("PAGE_5", genericRow("USER_2", 6L)) + .put("PAGE_5", genericRow("USER_3", 7L)) .build(); public PageViewDataProvider() { - super("PAGEVIEW", "VIEWTIME", PHYSICAL_SCHEMA, ROWS); + super("PAGEVIEW", PHYSICAL_SCHEMA, ROWS); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/TestDataProvider.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/TestDataProvider.java index 4a7d59c08b61..c2dc500da371 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/TestDataProvider.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/TestDataProvider.java @@ -15,30 +15,31 @@ package io.confluent.ksql.util; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.schema.ksql.Column.Namespace; import io.confluent.ksql.schema.ksql.PhysicalSchema; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.stream.Collectors; +import org.apache.kafka.streams.KeyValue; public abstract class TestDataProvider { private final String topicName; - private final String key; private final PhysicalSchema schema; - private final Map data; + private final Multimap data; private final String kstreamName; public TestDataProvider( final String namePrefix, - final String key, final PhysicalSchema schema, - final Map data + final Multimap data ) { this.topicName = Objects.requireNonNull(namePrefix, "namePrefix") + "_TOPIC"; this.kstreamName = namePrefix + "_KSTREAM"; - this.key = Objects.requireNonNull(key, "key"); this.schema = Objects.requireNonNull(schema, "schema"); this.data = Objects.requireNonNull(data, "data"); } @@ -54,17 +55,26 @@ public String ksqlSchemaString(final boolean asTable) { } public String key() { - return key; + return Iterables.getOnlyElement(schema.logicalSchema().key()).name().text(); } public PhysicalSchema schema() { return schema; } - public Map data() { + public Multimap data() { return data; } + public Map finalData() { + return data.entries().stream() + .collect(Collectors.toMap( + Entry::getKey, + Entry::getValue, + (first, second) -> second + )); + } + public String kstreamName() { return kstreamName; } @@ -78,4 +88,8 @@ private static String namespace(final Namespace namespace, final boolean asTable ? " PRIMARY KEY" : " KEY"; } + + static KeyValue kv(final K k, GenericRow v) { + return new KeyValue<>(k, v); + } } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/UserDataProvider.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/UserDataProvider.java index 9095f2e9a2d3..b2ea5a981f9d 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/UserDataProvider.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/UserDataProvider.java @@ -16,38 +16,37 @@ import static io.confluent.ksql.GenericRow.genericRow; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.Multimap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOption; -import java.util.Map; public class UserDataProvider extends TestDataProvider { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("USERID"), SqlTypes.STRING) .valueColumn(ColumnName.of("REGISTERTIME"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("GENDER"), SqlTypes.STRING) .valueColumn(ColumnName.of("REGIONID"), SqlTypes.STRING) - .valueColumn(ColumnName.of("USERID"), SqlTypes.STRING) .build(); private static final PhysicalSchema PHYSICAL_SCHEMA = PhysicalSchema .from(LOGICAL_SCHEMA, SerdeOption.none()); - private static final Map ROWS = ImmutableMap.builder() - .put("USER_0", genericRow(0L, "FEMALE", "REGION_0", "USER_0")) - .put("USER_1", genericRow(1L, "MALE", "REGION_1", "USER_1")) - .put("USER_2", genericRow(2L, "FEMALE", "REGION_1", "USER_2")) - .put("USER_3", genericRow(3L, "MALE", "REGION_0", "USER_3")) - .put("USER_4", genericRow(4L, "MALE", "REGION_4", "USER_4")) + private static final Multimap ROWS = ImmutableListMultimap + .builder() + .put("USER_0", genericRow(0L, "FEMALE", "REGION_0")) + .put("USER_1", genericRow(1L, "MALE", "REGION_1")) + .put("USER_2", genericRow(2L, "FEMALE", "REGION_1")) + .put("USER_3", genericRow(3L, "MALE", "REGION_0")) + .put("USER_4", genericRow(4L, "MALE", "REGION_4")) .build(); public UserDataProvider() { - super("USER", "USERID", PHYSICAL_SCHEMA, ROWS); + super("USER", PHYSICAL_SCHEMA, ROWS); } } \ No newline at end of file diff --git a/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index bd29b051a593..b7c9452d5153 100644 --- a/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -42,6 +42,7 @@ import java.util.function.BiConsumer; import java.util.function.Supplier; +@SuppressWarnings("UnstableApiUsage") public final class DataGen { private DataGen() { @@ -321,14 +322,6 @@ public String getTopicName(final Format format) { public String getKeyName() { return keyName; } - - public Format getKeyFormat() { - return FormatFactory.KAFKA; - } - - public Format getValueFormat() { - return FormatFactory.JSON; - } } Arguments build() { @@ -353,8 +346,8 @@ Arguments build() { if (quickstart != null) { schemaFile = Optional.ofNullable(schemaFile).orElse(quickstart.getSchemaFile()); - keyFormat = Optional.ofNullable(keyFormat).orElse(quickstart.getKeyFormat()); - valueFormat = Optional.ofNullable(valueFormat).orElse(quickstart.getValueFormat()); + keyFormat = Optional.ofNullable(keyFormat).orElse(FormatFactory.KAFKA); + valueFormat = Optional.ofNullable(valueFormat).orElse(FormatFactory.JSON); topicName = Optional.ofNullable(topicName).orElse(quickstart.getTopicName(valueFormat)); keyName = Optional.ofNullable(keyName).orElse(quickstart.getKeyName()); } @@ -386,19 +379,17 @@ Arguments build() { ); } - Builder parseArgs(final String[] args) throws IOException { + Builder parseArgs(final String[] args) { for (final String arg : args) { parseArg(arg); } return this; } - // CHECKSTYLE_RULES.OFF: CyclomaticComplexity - private Builder parseArg(final String arg) throws IOException { - // CHECKSTYLE_RULES.ON: CyclomaticComplexity + private void parseArg(final String arg) { if ("help".equals(arg) || "--help".equals(arg)) { help = true; - return this; + return; } final String[] splitOnEquals = arg.split("="); @@ -427,7 +418,6 @@ private Builder parseArg(final String arg) throws IOException { } setArg(argName, argValue); - return this; } private void setArg(final String argName, final String argVal) { diff --git a/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGenProducer.java b/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGenProducer.java index 3b1847c6f896..87868cf44c7a 100644 --- a/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGenProducer.java +++ b/ksqldb-examples/src/main/java/io/confluent/ksql/datagen/DataGenProducer.java @@ -35,6 +35,7 @@ import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Struct; +@SuppressWarnings("UnstableApiUsage") public class DataGenProducer { private final SerializerFactory keySerializerFactory; @@ -48,6 +49,7 @@ public DataGenProducer( this.valueSerializerFactory = requireNonNull(valueSerdeFactory, "valueSerdeFactory"); } + @SuppressWarnings("InfiniteLoopStatement") public void populateTopic( final Properties props, final Generator generator, @@ -102,7 +104,7 @@ public void populateTopic( producer.close(); } - private void produceOne( + private static void produceOne( final RowGenerator rowGenerator, final KafkaProducer producer, final String kafkaTopicName, diff --git a/ksqldb-examples/src/main/java/io/confluent/ksql/embedded/EmbeddedKsql.java b/ksqldb-examples/src/main/java/io/confluent/ksql/embedded/EmbeddedKsql.java index 7820d078c014..d196295d1fa9 100644 --- a/ksqldb-examples/src/main/java/io/confluent/ksql/embedded/EmbeddedKsql.java +++ b/ksqldb-examples/src/main/java/io/confluent/ksql/embedded/EmbeddedKsql.java @@ -33,9 +33,9 @@ public static void main(final String[] args) { ksqlContext.sql("REGISTER TOPIC orders_topic WITH (format = 'json', " + "kafka_topic='orders_topic_json');"); - ksqlContext.sql("CREATE STREAM orders (ordertime bigint, orderid bigint, itemid varchar, " + ksqlContext.sql("CREATE STREAM orders (orderid bigint KEY, ordertime bigint, itemid varchar, " + "orderunits double, arraycol array, mapcol map) " - + "WITH (topicname = 'orders_topic' , key='orderid');\n"); + + "WITH (topicname = 'orders_topic' );\n"); ksqlContext.sql("CREATE STREAM BIGORDERS AS SELECT * FROM ORDERS WHERE ORDERUNITS > 5;"); ksqlContext.sql("SELECT * FROM ORDERS;"); ksqlContext.sql("CREATE TABLE ORDERSUMS AS select itemid, sum(orderunits) from orders window " diff --git a/ksqldb-examples/src/main/resources/SampleQueries.sql b/ksqldb-examples/src/main/resources/SampleQueries.sql index 96889ba05533..c47495d087ea 100644 --- a/ksqldb-examples/src/main/resources/SampleQueries.sql +++ b/ksqldb-examples/src/main/resources/SampleQueries.sql @@ -2,8 +2,8 @@ REGISTER TOPIC users_topic WITH (value_format = 'json', kafka_topic='user_topic_ REGISTER TOPIC pageview_topic WITH (value_format = 'json', kafka_topic='pageview_topic_json'); CREATE STREAM pageview (viewtime bigint, pageid varchar, userid varchar) WITH (registered_topic = 'pageview_topic'); -CREATE TABLE users (registertime bigint, userid varchar, regionid varchar, gender varchar) WITH -(registered_topic = 'users_topic', KEY = 'userid'); +CREATE TABLE users (userid varchar PRIMARY KEY, registertime bigint, regionid varchar, gender varchar) WITH +(registered_topic = 'users_topic'); -- Enrich the pageview stream diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java index e01347dc34e7..9bcc1311b43a 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java @@ -17,14 +17,10 @@ import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.timestamp.TimestampColumn; -import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; -import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.SystemColumns; -import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.serde.WindowInfo; -import io.confluent.ksql.util.KsqlException; import java.util.Objects; import java.util.Optional; @@ -32,9 +28,9 @@ * Base class of create table/stream command */ public abstract class CreateSourceCommand implements DdlCommand { + private final SourceName sourceName; private final LogicalSchema schema; - private final Optional keyField; private final Optional timestampColumn; private final String topicName; private final Formats formats; @@ -43,7 +39,6 @@ public abstract class CreateSourceCommand implements DdlCommand { CreateSourceCommand( final SourceName sourceName, final LogicalSchema schema, - final Optional keyField, final Optional timestampColumn, final String topicName, final Formats formats, @@ -51,14 +46,13 @@ public abstract class CreateSourceCommand implements DdlCommand { ) { this.sourceName = Objects.requireNonNull(sourceName, "sourceName"); this.schema = Objects.requireNonNull(schema, "schema"); - this.keyField = Objects.requireNonNull(keyField, "keyField"); this.timestampColumn = Objects.requireNonNull(timestampColumn, "timestampColumn"); this.topicName = Objects.requireNonNull(topicName, "topicName"); this.formats = Objects.requireNonNull(formats, "formats"); this.windowInfo = Objects.requireNonNull(windowInfo, "windowInfo"); - validate(schema, keyField); + validate(schema); } public SourceName getSourceName() { @@ -73,10 +67,6 @@ public Optional getTimestampColumn() { return timestampColumn; } - public Optional getKeyField() { - return keyField; - } - public String getTopicName() { return topicName; } @@ -89,7 +79,7 @@ public Optional getWindowInfo() { return windowInfo; } - private void validate(final LogicalSchema schema, final Optional keyField) { + private static void validate(final LogicalSchema schema) { if (schema.valueContainsAny(SystemColumns.systemColumnNames())) { throw new IllegalArgumentException("Schema contains system columns in value schema"); } @@ -98,33 +88,6 @@ private void validate(final LogicalSchema schema, final Optional key throw new UnsupportedOperationException("Only single key columns supported. " + "Got: " + schema.key() + " (" + schema.key().size() + ")"); } - - if (keyField.isPresent()) { - final SqlType keyFieldType = schema.findColumn(keyField.get()) - .map(Column::type) - .orElseThrow(IllegalArgumentException::new); - - final SqlType keyType = schema.key().get(0).type(); - - if (!keyFieldType.equals(keyType)) { - final String primaryKey = this instanceof CreateTableCommand - ? " PRIMARY KEY" - : " KEY"; - - throw new KsqlException("The KEY field (" - + keyField.get().text() - + ") identified in the WITH clause is of a different type to the actual key column." - + System.lineSeparator() - + "Use of the KEY field is deprecated. Remove the KEY field from the WITH clause and " - + "specify the name of the key column by adding " - + "'" + keyField.get().text() + " " + keyFieldType + primaryKey + "' to the schema." - + System.lineSeparator() - + "KEY field type: " + keyFieldType - + System.lineSeparator() - + "key column type: " + keyType - ); - } - } } @Override @@ -138,7 +101,6 @@ public boolean equals(final Object o) { final CreateSourceCommand that = (CreateSourceCommand) o; return Objects.equals(sourceName, that.sourceName) && Objects.equals(schema, that.schema) - && Objects.equals(keyField, that.keyField) && Objects.equals(timestampColumn, that.timestampColumn) && Objects.equals(topicName, that.topicName) && Objects.equals(formats, that.formats) @@ -148,6 +110,6 @@ public boolean equals(final Object o) { @Override public int hashCode() { return Objects - .hash(sourceName, schema, keyField, timestampColumn, topicName, formats, windowInfo); + .hash(sourceName, schema, timestampColumn, topicName, formats, windowInfo); } } diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateStreamCommand.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateStreamCommand.java index 9e880244f328..d40ace8fdf87 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateStreamCommand.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateStreamCommand.java @@ -15,22 +15,23 @@ package io.confluent.ksql.execution.ddl.commands; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.timestamp.TimestampColumn; -import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.WindowInfo; import java.util.Optional; +@JsonIgnoreProperties({"keyField"}) // Removed after version 0.9 @Immutable public class CreateStreamCommand extends CreateSourceCommand { + public CreateStreamCommand( @JsonProperty(value = "sourceName", required = true) final SourceName sourceName, @JsonProperty(value = "schema", required = true) final LogicalSchema schema, - @JsonProperty(value = "keyField") final Optional keyField, @JsonProperty(value = "timestampColumn") final Optional timestampColumn, @JsonProperty(value = "topicName", required = true) final String topicName, @@ -40,7 +41,6 @@ public CreateStreamCommand( super( sourceName, schema, - keyField, timestampColumn, topicName, formats, diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java index 69fa6a1b2e80..c412a66be00f 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java @@ -15,22 +15,23 @@ package io.confluent.ksql.execution.ddl.commands; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.timestamp.TimestampColumn; -import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.WindowInfo; import java.util.Optional; +@JsonIgnoreProperties({"keyField"}) // Removed at version 0.10 @Immutable public class CreateTableCommand extends CreateSourceCommand { + public CreateTableCommand( @JsonProperty(value = "sourceName", required = true) final SourceName sourceName, @JsonProperty(value = "schema", required = true) final LogicalSchema schema, - @JsonProperty("keyField") final Optional keyField, @JsonProperty("timestampColumn") final Optional timestampColumn, @JsonProperty(value = "topicName", required = true) final String topicName, @JsonProperty(value = "formats", required = true) final Formats formats, @@ -39,7 +40,6 @@ public CreateTableCommand( super( sourceName, schema, - keyField, timestampColumn, topicName, formats, diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java index 2cf6fb2a8590..8599ecd7590d 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.WindowInfo; -import io.confluent.ksql.util.KsqlException; import java.util.Optional; import org.junit.Test; @@ -39,7 +38,6 @@ public class CreateSourceCommandTest { private static final Formats FORAMTS = mock(Formats.class); private static final ColumnName K0 = ColumnName.of("k0"); private static final ColumnName K1 = ColumnName.of("k1"); - private static final ColumnName KEY_FIELD = ColumnName.of("keyField"); @Test(expected = UnsupportedOperationException.class) @@ -57,72 +55,12 @@ public void shouldThrowOnMultipleKeyColumns() { SOURCE_NAME, schema, Optional.empty(), - Optional.empty(), TOPIC_NAME, FORAMTS, Optional.empty() ); } - @Test - public void shouldThrowIfKeyFieldDoesNotMatchKeyType() { - // Given: - final ColumnName keyField = ColumnName.of("keyField"); - - final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(K0, SqlTypes.INTEGER) - .valueColumn(keyField, SqlTypes.STRING) - .build(); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> new TestCommand( - SOURCE_NAME, - schema, - Optional.of(keyField), - Optional.empty(), - TOPIC_NAME, - FORAMTS, - Optional.empty() - ) - ); - - // Then: - assertThat(e.getMessage(), containsString( - "The KEY field (keyField) identified in the " - + "WITH clause is of a different type to the actual key column.")); - assertThat(e.getMessage(), containsString( - "Use of the KEY field is deprecated. Remove the KEY field from the WITH clause and " - + "specify the name of the key column by adding 'keyField STRING KEY' to the schema.")); - assertThat(e.getMessage(), containsString( - "KEY field type: STRING")); - assertThat(e.getMessage(), containsString( - "key column type: INTEGER")); - } - - @Test - public void shouldNotThrowIfKeyFieldMatchesKeyType() { - // Given: - final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(K0, SqlTypes.INTEGER) - .valueColumn(KEY_FIELD, SqlTypes.INTEGER) - .build(); - - // When: - new TestCommand( - SOURCE_NAME, - schema, - Optional.of(KEY_FIELD), - Optional.empty(), - TOPIC_NAME, - FORAMTS, - Optional.empty() - ); - - // Then: builds without error - } - @Test public void shouldThrowOnWindowStartColumn() { // Given: @@ -138,7 +76,6 @@ public void shouldThrowOnWindowStartColumn() { SOURCE_NAME, schema, Optional.empty(), - Optional.empty(), TOPIC_NAME, FORAMTS, Optional.empty() @@ -165,7 +102,6 @@ public void shouldThrowOnWindowEndColumn() { SOURCE_NAME, schema, Optional.empty(), - Optional.empty(), TOPIC_NAME, FORAMTS, Optional.empty() @@ -182,13 +118,12 @@ private static final class TestCommand extends CreateSourceCommand { TestCommand( final SourceName sourceName, final LogicalSchema schema, - final Optional keyField, final Optional timestampColumn, final String topicName, final Formats formats, final Optional windowInfo ) { - super(sourceName, schema, keyField, timestampColumn, topicName, formats, windowInfo); + super(sourceName, schema, timestampColumn, topicName, formats, windowInfo); } @Override diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/KeyFieldNode.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/KeyFieldNode.java deleted file mode 100644 index a91dea59cb34..000000000000 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/KeyFieldNode.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright 2019 Confluent Inc. - * - * Licensed under the Confluent Community License (the "License"); you may not use - * this file except in compliance with the License. You may obtain a copy of the - * License at - * - * http://www.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.test.model; - -import static java.util.Objects.requireNonNull; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; -import io.confluent.ksql.metastore.model.KeyField; -import io.confluent.ksql.test.model.matchers.MetaStoreMatchers.KeyFieldMatchers; -import java.util.Optional; -import org.hamcrest.Matcher; - -public final class KeyFieldNode { - - private static final KeyFieldNode NONE = new KeyFieldNode(Optional.empty()); - - @JsonValue - private final Optional name; - - @SuppressWarnings("WeakerAccess") // Invoked via reflection - @JsonCreator - public KeyFieldNode( - final Optional name - ) { - this.name = requireNonNull(name, "name"); - } - - /** - * @return a node that explicitly checks that no key field is set. - */ - public static KeyFieldNode none() { - return NONE; - } - - Matcher build() { - return KeyFieldMatchers.hasName(name); - } -} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java index e703bc37c68b..9cae8b2b3a47 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java @@ -45,7 +45,6 @@ final class SourceNode { private final String name; private final String type; - private final Optional keyField; private final Optional schema; private final Optional keyFormat; @@ -53,13 +52,11 @@ final class SourceNode { SourceNode( final String name, final String type, - final Optional keyField, final Optional schema, final Optional keyFormat ) { this.name = Objects.requireNonNull(name, "name"); this.type = Objects.requireNonNull(type, "type"); - this.keyField = Objects.requireNonNull(keyField, "keyField"); this.schema = Objects.requireNonNull(schema, "schema"); this.keyFormat = Objects.requireNonNull(keyFormat, "keyFormat"); @@ -99,11 +96,6 @@ Matcher build() { final Matcher typeMatcher = IsInstanceOf .instanceOf(toType(type)); - final Matcher keyFieldMatcher = keyField - .map(KeyFieldNode::build) - .map(MetaStoreMatchers::hasKeyField) - .orElse(null); - final Matcher schemaMatcher = schema .map(SourceNode::parseSchema) .map(Matchers::is) @@ -116,7 +108,7 @@ Matcher build() { .orElse(null); final Matcher[] matchers = Stream - .of(nameMatcher, typeMatcher, keyFieldMatcher, schemaMatcher, keyFormatMatcher) + .of(nameMatcher, typeMatcher, schemaMatcher, keyFormatMatcher) .filter(Objects::nonNull) .toArray(Matcher[]::new); @@ -153,16 +145,13 @@ public SourceNode deserialize( final String name = JsonParsingUtil.getRequired("name", node, jp, String.class); final String type = JsonParsingUtil.getRequired("type", node, jp, String.class); - final Optional keyField = JsonParsingUtil - .getOptionalOrElse("keyField", node, jp, KeyFieldNode.class, KeyFieldNode.none()); - final Optional rawSchema = JsonParsingUtil .getOptional("schema", node, jp, String.class); final Optional keyFormat = JsonParsingUtil .getOptional("keyFormat", node, jp, KeyFormatNode.class); - return new SourceNode(name, type, keyField, rawSchema, keyFormat); + return new SourceNode(name, type, rawSchema, keyFormat); } } } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/matchers/MetaStoreMatchers.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/matchers/MetaStoreMatchers.java index c564c023a1b1..798ce4698326 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/matchers/MetaStoreMatchers.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/matchers/MetaStoreMatchers.java @@ -18,13 +18,9 @@ import static org.hamcrest.Matchers.is; import io.confluent.ksql.metastore.model.DataSource; -import io.confluent.ksql.metastore.model.KeyField; -import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; -import io.confluent.ksql.parser.ColumnReferenceParser; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.KeyFormat; -import java.util.Optional; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -46,21 +42,6 @@ protected SourceName featureValueOf(final DataSource actual) { }; } - public static Matcher hasKeyField( - final Matcher fieldMatcher - ) { - return new FeatureMatcher( - fieldMatcher, - "source with key field", - "key field" - ) { - @Override - protected KeyField featureValueOf(final DataSource actual) { - return actual.getKeyField(); - } - }; - } - public static Matcher hasSchema( final Matcher schemaMatcher ) { @@ -89,23 +70,4 @@ protected KeyFormat featureValueOf(final DataSource actual) { } }; } - - public static final class KeyFieldMatchers { - - private KeyFieldMatchers() { - } - - public static Matcher hasName(final Optional name) { - return new FeatureMatcher>( - is(name.map(ColumnReferenceParser::parse)), - "field with name", - "name" - ) { - @Override - protected Optional featureValueOf(final KeyField actual) { - return actual.ref(); - } - }; - } - } } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/conditions/PostConditions.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/conditions/PostConditions.java index 3deb1da50f07..0df68f32f6f0 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/conditions/PostConditions.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/conditions/PostConditions.java @@ -107,7 +107,6 @@ private void verifyMetaStore(final MetaStore metaStore) { final String text = values.stream() .map(s -> s.getDataSourceType() + ":" + s.getName().text() - + ", key:" + s.getKeyField().ref() + ", value:" + s.getSchema() + ", keyFormat:" + s.getKsqlTopic().getKeyFormat() ) diff --git a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/model/SourceNodeTest.java b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/model/SourceNodeTest.java index fd4ba348059e..42944b4d55f4 100644 --- a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/model/SourceNodeTest.java +++ b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/model/SourceNodeTest.java @@ -23,7 +23,6 @@ public class SourceNodeTest { static final SourceNode INSTANCE = new SourceNode( "bob", "stream", - Optional.empty(), Optional.of("ROWKEY INT KEY, NAME STRING"), Optional.of(KeyFormatNodeTest.INSTANCE) ); diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/plan.json new file mode 100644 index 000000000000..6c159c965bbf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (K INTEGER PRIMARY KEY, USER INTEGER, REGION STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`K` INTEGER KEY, `USER` INTEGER, `REGION` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n 1 KSQL_COL_0,\n COUNT(*) KSQL_COL_1\nFROM TEST TEST\nGROUP BY 1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_0` INTEGER KEY, `KSQL_COL_1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`K` INTEGER KEY, `USER` INTEGER, `REGION` STRING" + }, + "selectExpressions" : [ "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/spec.json new file mode 100644 index 000000000000..0573faf99edd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/spec.json @@ -0,0 +1,117 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386990638, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "constant (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,r0" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r1" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,r0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "3" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "2" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (K INT PRIMARY KEY, user INT, region VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT 1, COUNT(*) FROM TEST GROUP BY 1;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_constant_(table-_table)/6.0.0_1589386990638/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/plan.json new file mode 100644 index 000000000000..d356cb623a95 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, IGNORED STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `IGNORED` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.K K,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`K` STRING KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "K AS K", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "K", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/spec.json new file mode 100644 index 000000000000..fc15ed5bd270 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/spec.json @@ -0,0 +1,133 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386987411, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : "-" + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : "-" + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : "-" + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : "-" + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : "-" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,3" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "3" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT K, COUNT(*) FROM TEST GROUP BY K;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "K STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1589386987411/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/plan.json new file mode 100644 index 000000000000..6ffed7cb1780 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, IGNORED STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `IGNORED` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.K K,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`K` STRING KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "K AS K", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "nonAggregateColumns" : [ "K", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/spec.json new file mode 100644 index 000000000000..86c96b694e4a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/spec.json @@ -0,0 +1,190 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386987557, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field (stream->table) - format - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "IGNORED", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE TABLE OUTPUT AS SELECT K, COUNT(*) FROM TEST GROUP BY K;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "K STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1589386987557/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/plan.json new file mode 100644 index 000000000000..aa2afabec62e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, IGNORED STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `IGNORED` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.K K,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "K AS K", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "K", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/spec.json new file mode 100644 index 000000000000..51f6bef9c733 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/spec.json @@ -0,0 +1,178 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386987645, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field (stream->table) - format - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT K, COUNT(*) FROM TEST GROUP BY K;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "K STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1589386987645/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/plan.json new file mode 100644 index 000000000000..456a9dd73f33 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, IGNORED STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `IGNORED` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.K K,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`K` STRING KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "K AS K", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "nonAggregateColumns" : [ "K", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/spec.json new file mode 100644 index 000000000000..0831a48341d8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/spec.json @@ -0,0 +1,180 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386987762, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field (stream->table) - format - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "K" : "d2", + "ROWTIME" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "K" : "d1", + "ROWTIME" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string IGNORED = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE TABLE OUTPUT AS SELECT K, COUNT(*) FROM TEST GROUP BY K;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "K STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386987762/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/plan.json new file mode 100644 index 000000000000..c41d83eabcec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, USER INTEGER, REGION STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.REGION REGION,\n SUBSTRING(TEST.REGION, 2, 1) KSQL_COL_0,\n COUNT(*) KSQL_COL_1\nFROM TEST TEST\nGROUP BY TEST.REGION\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`REGION` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING" + }, + "selectExpressions" : [ "REGION AS REGION", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "REGION" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "REGION", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "SUBSTRING(REGION, 2, 1) AS KSQL_COL_0", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/spec.json new file mode 100644 index 000000000000..414d44e43cff --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/spec.json @@ -0,0 +1,117 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386990851, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field with field used in function in projection (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,r0" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r1" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,r0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "0,1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "1,1" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "0,2" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "0,1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "1,0" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "0,2" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, user INT, region VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT region, SUBSTRING(region, 2, 1), COUNT(*) FROM TEST GROUP BY region;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_field_used_in_function_in_projection_(table-_table)/6.0.0_1589386990851/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/plan.json new file mode 100644 index 000000000000..f8587d6da0e4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, USER INTEGER, REGION STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.REGION REGION,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.REGION\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`REGION` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING" + }, + "selectExpressions" : [ "REGION AS REGION", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "REGION" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "REGION", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/spec.json new file mode 100644 index 000000000000..a90222134b9c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/spec.json @@ -0,0 +1,122 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386989345, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "field with re-key (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,r0" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r1" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,r0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, user INT, region VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT region, COUNT(*) FROM TEST GROUP BY region;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "REGION STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(table-_table)/6.0.0_1589386989345/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/plan.json new file mode 100644 index 000000000000..a56c32eca7db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/spec.json new file mode 100644 index 000000000000..8c9c7d3671ea --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/spec.json @@ -0,0 +1,143 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386987914, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,a" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : "3,a,0,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : "1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1589386987914/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/plan.json new file mode 100644 index 000000000000..68d5dc5ee8fa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/spec.json new file mode 100644 index 000000000000..0694a231c4aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/spec.json @@ -0,0 +1,209 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988079, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (stream->table) - format - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "F1", + "type" : [ "null", "int" ], + "default" : null + }, { + "name" : "F2", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1589386988079/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/plan.json new file mode 100644 index 000000000000..45e6a55cacc3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/spec.json new file mode 100644 index 000000000000..6b5ef00382e0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/spec.json @@ -0,0 +1,193 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988168, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (stream->table) - format - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1589386988168/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/plan.json new file mode 100644 index 000000000000..a3f26dabddbb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/spec.json new file mode 100644 index 000000000000..26122f69ff4c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/spec.json @@ -0,0 +1,195 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988249, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (stream->table) - format - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 F1 = 1;\n string F2 = 2;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1589386988249/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/plan.json new file mode 100644 index 000000000000..09ccada1084f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/spec.json new file mode 100644 index 000000000000..9121e9e39a7c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/spec.json @@ -0,0 +1,159 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988413, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,b" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1589386988413/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/plan.json new file mode 100644 index 000000000000..21f1320aae42 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/spec.json new file mode 100644 index 000000000000..0ccd68b29105 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/spec.json @@ -0,0 +1,236 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988647, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (table->table) - format - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "F1", + "type" : [ "null", "int" ], + "default" : null + }, { + "name" : "F2", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1589386988647/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/plan.json new file mode 100644 index 000000000000..3f0a2825c62c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/spec.json new file mode 100644 index 000000000000..c39468f9baac --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/spec.json @@ -0,0 +1,220 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988760, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (table->table) - format - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1589386988760/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/plan.json new file mode 100644 index 000000000000..49dabf132c13 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/spec.json new file mode 100644 index 000000000000..ee41993dde8c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/spec.json @@ -0,0 +1,222 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988869, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields (table->table) - format - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "ROWTIME" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 F1 = 1;\n string F2 = 2;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE TABLE OUTPUT AS SELECT f1, f2, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1589386988869/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/plan.json new file mode 100644 index 000000000000..b03bf1f83a54 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n AS_VALUE(TEST.F1) F3,\n AS_VALUE(TEST.F2) F4,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `F3` INTEGER, `F4` STRING, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "AS_VALUE(F1) AS F3", "AS_VALUE(F2) AS F4", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/spec.json new file mode 100644 index 000000000000..2e52fa6344c8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/spec.json @@ -0,0 +1,159 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988518, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "fields - copied into value (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,b" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "1,b,1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "1,b,0" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f1 INT, f2 VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, f2, AS_VALUE(f1) AS F3, AS_VALUE(F2) AS F4, COUNT(*) FROM TEST GROUP BY f2, f1;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, F3 INT, F4 STRING, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_-_copied_into_value_(table-_table)/6.0.0_1589386988518/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/plan.json new file mode 100644 index 000000000000..74732f9dcba8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, USER INTEGER, REGION STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n SUBSTRING(TEST.REGION, 7, 2) KSQL_COL_0,\n COUNT(*) KSQL_COL_1\nFROM TEST TEST\nGROUP BY SUBSTRING(TEST.REGION, 7, 2)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_0` STRING KEY, `KSQL_COL_1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING" + }, + "selectExpressions" : [ "REGION AS REGION", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "SUBSTRING(REGION, 7, 2)" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "REGION", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/spec.json new file mode 100644 index 000000000000..d9af03d88ad2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/spec.json @@ -0,0 +1,122 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386990102, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "function (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,prefixr0" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,prefixr1" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,prefixr0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,prefixr0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, user INT, region VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT SUBSTRING(region, 7, 2), COUNT(*) FROM TEST GROUP BY SUBSTRING(region, 7, 2);" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_0 STRING KEY, KSQL_COL_1 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_(table-_table)/6.0.0_1589386990102/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/plan.json new file mode 100644 index 000000000000..88bfc028ad19 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, USER INTEGER, REGION STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.REGION REGION,\n (COUNT(*) * 2) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.REGION\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`REGION` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `USER` INTEGER, `REGION` STRING" + }, + "selectExpressions" : [ "REGION AS REGION", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "REGION" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "REGION", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "(KSQL_AGG_VARIABLE_0 * 2) AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/spec.json new file mode 100644 index 000000000000..9e8c7161bc2c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/spec.json @@ -0,0 +1,122 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386989597, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "with aggregate arithmetic (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,r0" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r1" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,r0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,r0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "4" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "r1", + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : "r0", + "value" : "4" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, user INT, region VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT region, COUNT(*) * 2 FROM TEST GROUP BY region;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "REGION STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(table-_table)/6.0.0_1589386989597/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/plan.json new file mode 100644 index 000000000000..8bf5b8f4a0c8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID INTEGER PRIMARY KEY, F0 INTEGER, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F0` INTEGER, `F1` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F0 F0,\n (TEST.F0 * SUM(TEST.F1)) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F0\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`F0` INTEGER KEY, `KSQL_COL_0` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F0` INTEGER, `F1` INTEGER" + }, + "selectExpressions" : [ "F0 AS F0", "F1 AS F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F0", "F1" ], + "aggregationFunctions" : [ "SUM(F1)" ] + }, + "selectExpressions" : [ "(F0 * KSQL_AGG_VARIABLE_0) AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/spec.json new file mode 100644 index 000000000000..aee797a3f305 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/spec.json @@ -0,0 +1,113 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386989894, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "with aggregate arithmetic involving source field (table->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 2, + "value" : "2,10" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,20" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,30" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : "20" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "40" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "0" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "60" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "0" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID INT PRIMARY KEY, f0 INT, f1 INT) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f0, f0 * SUM(f1) FROM TEST GROUP BY f0;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(table-_table)/6.0.0_1589386989894/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/plan.json new file mode 100644 index 000000000000..a28813e8cb22 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n 'some constant' F3,\n COUNT(TEST.F1) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`F1` INTEGER KEY, `F3` STRING, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER" + }, + "selectExpressions" : [ "F1 AS F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1" ], + "aggregationFunctions" : [ "COUNT(F1)" ] + }, + "selectExpressions" : [ "'some constant' AS F3", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/spec.json new file mode 100644 index 000000000000..528784586669 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/spec.json @@ -0,0 +1,113 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386991753, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "with constants in the projection (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : "some constant,1" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "some constant,1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "some constant,2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "some constant,2" + }, { + "topic" : "OUTPUT", + "key" : 3, + "value" : "some constant,1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, 'some constant' as f3, COUNT(f1) FROM TEST GROUP BY f1;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1589386991753/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/plan.json new file mode 100644 index 000000000000..385bc9a98964 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/plan.json @@ -0,0 +1,169 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nHAVING (SUM(TEST.F1) > 1)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`F1` INTEGER KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER" + }, + "selectExpressions" : [ "F1 AS F1", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)", "SUM(F1)" ] + }, + "filterExpression" : "(KSQL_AGG_VARIABLE_1 > 1)" + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/spec.json new file mode 100644 index 000000000000..47f11ca67512 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/spec.json @@ -0,0 +1,113 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386991385, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "with having expression (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 3, + "value" : "1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, COUNT(*) FROM TEST GROUP BY f1 HAVING SUM(f1) > 1;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1589386991385/topology @@ -0,0 +1,49 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000014 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000014 (stores: []) + --> KSTREAM-SINK-0000000015 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000015 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000014 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/plan.json new file mode 100644 index 000000000000..33282f8bbaa1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/plan.json @@ -0,0 +1,162 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, F1 INTEGER, F2 STRING, F3 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING, `F3` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n TEST.F3 F3,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F3, TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `F1` INTEGER, `F2` STRING, `F3` INTEGER" + }, + "selectExpressions" : [ "F1 AS F1", "F2 AS F2", "F3 AS F3", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "groupByExpressions" : [ "F3", "F2", "F1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "F1", "F2", "F3", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/spec.json new file mode 100644 index 000000000000..80082bb84032 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/spec.json @@ -0,0 +1,118 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386988327, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "with single grouping set (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a,-1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b,-2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a,-1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b,-2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,a,-3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "-1|+|a|+|1", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "-2|+|b|+|2", + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : "-1|+|a|+|1", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "-2|+|b|+|2", + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : "-3|+|a|+|3", + "value" : "1" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID INT KEY, f1 INT, f2 VARCHAR, f3 INT) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE OUTPUT AS SELECT f1, f2, f3, COUNT(*) FROM TEST GROUP BY (f3, f2, f1);" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "KSQL_COL_1 STRING KEY, KSQL_COL_0 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_single_grouping_set_(stream-_table)/6.0.0_1589386988327/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/plan.json new file mode 100644 index 000000000000..f065f52f2832 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/plan.json @@ -0,0 +1,168 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE AVG AS SELECT\n TEST.ID ID,\n (SUM(TEST.VALUE) / COUNT(TEST.ID)) AVG\nFROM TEST TEST\nGROUP BY TEST.ID\nHAVING ((SUM(TEST.VALUE) / COUNT(TEST.ID)) > 25)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "AVG", + "schema" : "`ID` BIGINT KEY, `AVG` BIGINT", + "topicName" : "AVG", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "AVG", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "AVG" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "SUM(VALUE)", "COUNT(ID)", "SUM(VALUE)", "COUNT(ID)" ] + }, + "filterExpression" : "((KSQL_AGG_VARIABLE_2 / KSQL_AGG_VARIABLE_3) > 25)" + }, + "selectExpressions" : [ "(KSQL_AGG_VARIABLE_0 / KSQL_AGG_VARIABLE_1) AS AVG" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "AVG" + }, + "queryId" : "CTAS_AVG_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/spec.json new file mode 100644 index 000000000000..7373561c693c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/spec.json @@ -0,0 +1,103 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993127, + "path" : "query-validation-tests/having.json", + "schemas" : { + "CTAS_AVG_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_AVG_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_AVG_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_AVG_0.AVG" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "calculate average in having", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,50" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,10" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,15" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "one,100" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "one,10" + } ], + "outputs" : [ { + "topic" : "AVG", + "key" : 0, + "value" : "50" + }, { + "topic" : "AVG", + "key" : 0, + "value" : "30" + }, { + "topic" : "AVG", + "key" : 0, + "value" : null + }, { + "topic" : "AVG", + "key" : 1, + "value" : "100" + }, { + "topic" : "AVG", + "key" : 1, + "value" : "55" + } ], + "topics" : [ { + "name" : "AVG", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE AVG AS select id, sum(value)/count(id) as avg from test GROUP BY id HAVING sum(value)/count(id)> 25;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "AVG", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_AVG_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/topology new file mode 100644 index 000000000000..8a6f56ea7dca --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_calculate_average_in_having/6.0.0_1589386993127/topology @@ -0,0 +1,34 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000009 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000009 (stores: []) + --> KSTREAM-SINK-0000000010 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000010 (topic: AVG) + <-- KTABLE-TOSTREAM-0000000009 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/plan.json new file mode 100644 index 000000000000..b205e3a05dd6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/plan.json @@ -0,0 +1,173 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 AS SELECT\n TEST.ID ID,\n SUM(TEST.VALUE) SUM\nFROM TEST TEST\nWINDOW TUMBLING ( SIZE 30 SECONDS ) \nGROUP BY TEST.ID\nHAVING (SUM(TEST.VALUE) > 100)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ID` BIGINT KEY, `SUM` BIGINT", + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "T1", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "T1" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "SUM(VALUE)", "SUM(VALUE)" ], + "windowExpression" : " TUMBLING ( SIZE 30 SECONDS ) " + }, + "filterExpression" : "(KSQL_AGG_VARIABLE_1 > 100)" + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS SUM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "T1" + }, + "queryId" : "CTAS_T1_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/spec.json new file mode 100644 index 000000000000..9939c6cf2698 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993055, + "path" : "query-validation-tests/having.json", + "schemas" : { + "CTAS_T1_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_T1_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_T1_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_T1_0.T1" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "table having", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,0" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "one,100" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "one,5" + } ], + "outputs" : [ { + "topic" : "T1", + "key" : 0, + "value" : null, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "T1", + "key" : 1, + "value" : null, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "T1", + "key" : 1, + "value" : "105", + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "T1", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE T1 as select id, sum(value) as sum from test WINDOW TUMBLING (SIZE 30 SECONDS) group by id HAVING sum(value) > 100;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "T1", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_T1_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/topology new file mode 100644 index 000000000000..a4c2607c2964 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/having_-_table_having/6.0.0_1589386993055/topology @@ -0,0 +1,37 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-WindowSelect + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000010 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000010 (stores: []) + --> KSTREAM-SINK-0000000011 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000011 (topic: T1) + <-- KTABLE-TOSTREAM-0000000010 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/plan.json new file mode 100644 index 000000000000..6096e43b24da --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n HISTOGRAM(TEST.VALUE) COUNTS\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "HISTOGRAM(VALUE)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNTS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/spec.json new file mode 100644 index 000000000000..04fe55a6bf90 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/spec.json @@ -0,0 +1,147 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993189, + "path" : "query-validation-tests/histogram.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT> NOT NULL", + "CTAS_S2_0.S2" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "histogram string - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "foo" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "bar" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "foo" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 1 + } + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1, + "bar" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2, + "foo" : 1 + } + } + } ], + "topics" : [ { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, histogram(value) as counts FROM test group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_AVRO/6.0.0_1589386993189/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/plan.json new file mode 100644 index 000000000000..f6e631ffbf46 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n HISTOGRAM(TEST.VALUE) COUNTS\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "HISTOGRAM(VALUE)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNTS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/spec.json new file mode 100644 index 000000000000..21197fe16940 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/spec.json @@ -0,0 +1,135 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993254, + "path" : "query-validation-tests/histogram.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT> NOT NULL", + "CTAS_S2_0.S2" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "histogram string - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "foo" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "bar" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "foo" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 1 + } + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1, + "bar" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2, + "foo" : 1 + } + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE S2 as SELECT ID, histogram(value) as counts FROM test group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_JSON/6.0.0_1589386993254/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/plan.json new file mode 100644 index 000000000000..49210403f5ff --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/plan.json @@ -0,0 +1,161 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n HISTOGRAM(TEST.VALUE) COUNTS\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "HISTOGRAM(VALUE)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNTS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/spec.json new file mode 100644 index 000000000000..ec24ecbf5a69 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/spec.json @@ -0,0 +1,137 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993323, + "path" : "query-validation-tests/histogram.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT> NOT NULL", + "CTAS_S2_0.S2" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "histogram string - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "foo" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "bar" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "foo" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 1 + } + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1, + "bar" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2, + "foo" : 1 + } + } + } ], + "topics" : [ { + "name" : "test_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string VALUE = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE TABLE S2 as SELECT ID, histogram(value) as counts FROM test group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF/6.0.0_1589386993323/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/plan.json new file mode 100644 index 000000000000..6f81bf51ea9c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/plan.json @@ -0,0 +1,167 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, ID INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `ID` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COUNT(1) COUNT\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 5 SECONDS , ADVANCE BY 1 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `ID` INTEGER" + }, + "selectExpressions" : [ "ID AS ID", "1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "ID" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " HOPPING ( SIZE 5 SECONDS , ADVANCE BY 1 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/spec.json new file mode 100644 index 000000000000..b50258bf27cf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/spec.json @@ -0,0 +1,219 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386994011, + "path" : "query-validation-tests/hopping-windows.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "count", + "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" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE S2 as SELECT ID, count(1) as count FROM test WINDOW HOPPING (SIZE 5 SECOND, ADVANCE BY 1 SECONDS) group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/topology new file mode 100644 index 000000000000..d2b377a3888c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_count/6.0.0_1589386994011/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000013 (topic: S2) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/plan.json new file mode 100644 index 000000000000..51b2df1c458f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/plan.json @@ -0,0 +1,166 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `KSQL_COL_0` BIGINT", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MAX(VALUE)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/spec.json new file mode 100644 index 000000000000..0c0c7532ec55 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/spec.json @@ -0,0 +1,295 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993744, + "path" : "query-validation-tests/hopping-windows.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "max hopping", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,6", + "timestamp" : 45000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,300", + "timestamp" : 50000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,100", + "timestamp" : 35000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "100,2000", + "timestamp" : 40000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "5", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "5", + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 45000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 45000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "6", + "timestamp" : 45000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "300", + "timestamp" : 50000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "300", + "timestamp" : 50000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "300", + "timestamp" : 50000, + "window" : { + "start" : 50000, + "end" : 80000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "100", + "timestamp" : 35000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "100", + "timestamp" : 35000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "100", + "timestamp" : 35000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "2000", + "timestamp" : 40000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "2000", + "timestamp" : 40000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "2000", + "timestamp" : 40000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE S2 as SELECT ID, max(value) FROM test WINDOW HOPPING (SIZE 30 SECONDS, ADVANCE BY 10 SECONDS) group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "table", + "keyFormat" : { + "format" : "KAFKA", + "windowType" : "HOPPING", + "windowSize" : 30000 + } + } ], + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1589386993744/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/plan.json new file mode 100644 index 000000000000..5a8baaf2bbf2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/plan.json @@ -0,0 +1,166 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MIN(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `KSQL_COL_0` BIGINT", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MIN(VALUE)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/spec.json new file mode 100644 index 000000000000..c47cce33ebe0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/spec.json @@ -0,0 +1,146 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993815, + "path" : "query-validation-tests/hopping-windows.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "min hopping", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "5", + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE TABLE S2 as SELECT ID, min(value) FROM test WINDOW HOPPING (SIZE 30 SECONDS, ADVANCE BY 10 SECONDS) group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1589386993815/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/plan.json new file mode 100644 index 000000000000..bebd9e70781f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/plan.json @@ -0,0 +1,166 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE DOUBLE) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPK(TEST.VALUE, 2) TOPK\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "2 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPK(VALUE, 2)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/spec.json new file mode 100644 index 000000000000..3ef712400090 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/spec.json @@ -0,0 +1,123 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993890, + "path" : "query-validation-tests/hopping-windows.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT> NOT NULL", + "CTAS_S2_0.S2" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "topk hopping", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : 100 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 0.0 ] + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0, 0.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE double) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE S2 as SELECT ID, topk(value, 2) as topk FROM test WINDOW HOPPING (SIZE 30 SECONDS, ADVANCE BY 10 SECONDS) group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topk_hopping/6.0.0_1589386993890/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/plan.json new file mode 100644 index 000000000000..3dfdd5fd593f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/plan.json @@ -0,0 +1,166 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE DOUBLE) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPKDISTINCT(TEST.VALUE, 2) TOPK\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "2 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPKDISTINCT(VALUE, 2)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "S2" + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/spec.json new file mode 100644 index 000000000000..a387ca5d468a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/spec.json @@ -0,0 +1,155 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386993956, + "path" : "query-validation-tests/hopping-windows.json", + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT> NOT NULL", + "CTAS_S2_0.S2" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "topkdistinct hopping", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : 100 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 0.0 ] + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0, 0.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0, 0.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100.0 ] + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE double) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE S2 as SELECT ID, topkdistinct(value, 2) as topk FROM test WINDOW HOPPING (SIZE 30 SECONDS, ADVANCE BY 10 SECONDS) group by id;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_topkdistinct_hopping/6.0.0_1589386993956/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/plan.json new file mode 100644 index 000000000000..603b1fc21a63 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/plan.json @@ -0,0 +1,197 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_1 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_1", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_2 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_2", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n I1.FOO I1_FOO,\n I1.BAR I1_BAR,\n I2.BAR I2_BAR\nFROM INPUT_1 I1\nINNER JOIN INPUT_2 I2 WITHIN 1 MINUTES ON ((I1.FOO = I2.FOO))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`I1_FOO` INTEGER KEY, `I1_BAR` INTEGER, `I2_BAR` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "INPUT_1", "INPUT_2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I1_BAR", "ROWTIME AS I1_ROWTIME", "FOO AS I1_FOO" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I2_BAR", "ROWTIME AS I2_ROWTIME", "FOO AS I2_FOO" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000, + "keyColName" : "I1_FOO" + }, + "selectExpressions" : [ "I1_BAR AS I1_BAR", "I2_BAR AS I2_BAR" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/spec.json new file mode 100644 index 000000000000..7e9144d9ee10 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/spec.json @@ -0,0 +1,108 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386994151, + "path" : "query-validation-tests/identifiers.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "aliased join source", + "inputs" : [ { + "topic" : "t1", + "key" : 1, + "value" : { + "bar" : 2 + } + }, { + "topic" : "t2", + "key" : 1, + "value" : { + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "I1_BAR" : 2, + "I2_BAR" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT_1 (FOO INT KEY, bar INT) WITH (kafka_topic='t1', value_format='JSON');", "CREATE STREAM INPUT_2 (FOO INT KEY, bar INT) WITH (kafka_topic='t2', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT I1.FOO, I1.BAR, I2.BAR FROM INPUT_1 I1 JOIN INPUT_2 I2 WITHIN 1 MINUTE ON I1.FOO = I2.FOO;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "t1", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "t2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/topology new file mode 100644 index 000000000000..b39e415a7480 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source/6.0.0_1589386994151/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [t2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/plan.json new file mode 100644 index 000000000000..603b1fc21a63 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/plan.json @@ -0,0 +1,197 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_1 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_1", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_2 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_2", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n I1.FOO I1_FOO,\n I1.BAR I1_BAR,\n I2.BAR I2_BAR\nFROM INPUT_1 I1\nINNER JOIN INPUT_2 I2 WITHIN 1 MINUTES ON ((I1.FOO = I2.FOO))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`I1_FOO` INTEGER KEY, `I1_BAR` INTEGER, `I2_BAR` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "INPUT_1", "INPUT_2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I1_BAR", "ROWTIME AS I1_ROWTIME", "FOO AS I1_FOO" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I2_BAR", "ROWTIME AS I2_ROWTIME", "FOO AS I2_FOO" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000, + "keyColName" : "I1_FOO" + }, + "selectExpressions" : [ "I1_BAR AS I1_BAR", "I2_BAR AS I2_BAR" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/spec.json new file mode 100644 index 000000000000..15d1bb049acd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/spec.json @@ -0,0 +1,108 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386994217, + "path" : "query-validation-tests/identifiers.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "aliased join source with AS", + "inputs" : [ { + "topic" : "t1", + "key" : 1, + "value" : { + "bar" : 2 + } + }, { + "topic" : "t2", + "key" : 1, + "value" : { + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "I1_BAR" : 2, + "I2_BAR" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT_1 (FOO INT KEY, bar INT) WITH (kafka_topic='t1', value_format='JSON');", "CREATE STREAM INPUT_2 (FOO INT KEY, bar INT) WITH (kafka_topic='t2', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT I1.FOO, I1.BAR, I2.BAR FROM INPUT_1 AS I1 JOIN INPUT_2 AS I2 WITHIN 1 MINUTE ON I1.FOO = I2.FOO;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "t1", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "t2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/topology new file mode 100644 index 000000000000..b39e415a7480 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_join_source_with_AS/6.0.0_1589386994217/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [t2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/plan.json new file mode 100644 index 000000000000..197c0a38cdc1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/plan.json @@ -0,0 +1,197 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_1 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_1", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_2 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_2", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n I1.FOO I1_FOO,\n I1.BAR I1_BAR,\n INPUT_2.BAR INPUT_2_BAR\nFROM INPUT_1 I1\nINNER JOIN INPUT_2 INPUT_2 WITHIN 1 MINUTES ON ((I1.FOO = INPUT_2.FOO))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`I1_FOO` INTEGER KEY, `I1_BAR` INTEGER, `INPUT_2_BAR` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "INPUT_1", "INPUT_2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I1_BAR", "ROWTIME AS I1_ROWTIME", "FOO AS I1_FOO" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS INPUT_2_BAR", "ROWTIME AS INPUT_2_ROWTIME", "FOO AS INPUT_2_FOO" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000, + "keyColName" : "I1_FOO" + }, + "selectExpressions" : [ "I1_BAR AS I1_BAR", "INPUT_2_BAR AS INPUT_2_BAR" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/spec.json new file mode 100644 index 000000000000..8bb44af265fc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/spec.json @@ -0,0 +1,108 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386994303, + "path" : "query-validation-tests/identifiers.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "aliased left unaliased right", + "inputs" : [ { + "topic" : "t1", + "key" : 1, + "value" : { + "bar" : 2 + } + }, { + "topic" : "t2", + "key" : 1, + "value" : { + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "I1_BAR" : 2, + "INPUT_2_BAR" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT_1 (FOO INT KEY, bar INT) WITH (kafka_topic='t1', value_format='JSON');", "CREATE STREAM INPUT_2 (FOO INT KEY, bar INT) WITH (kafka_topic='t2', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT I1.FOO, I1.BAR, INPUT_2.BAR FROM INPUT_1 AS I1 JOIN INPUT_2 WITHIN 1 MINUTE ON I1.FOO = INPUT_2.FOO;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "t1", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "t2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/topology new file mode 100644 index 000000000000..b39e415a7480 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_aliased_left_unaliased_right/6.0.0_1589386994303/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [t2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/plan.json new file mode 100644 index 000000000000..57d8f006594f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/plan.json @@ -0,0 +1,197 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_1 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_1", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_2 (FOO INTEGER KEY, BAR INTEGER) WITH (KAFKA_TOPIC='t2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_2", + "schema" : "`FOO` INTEGER KEY, `BAR` INTEGER", + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT_1.FOO INPUT_1_FOO,\n INPUT_1.BAR INPUT_1_BAR,\n I2.BAR I2_BAR\nFROM INPUT_1 INPUT_1\nINNER JOIN INPUT_2 I2 WITHIN 1 MINUTES ON ((INPUT_1.FOO = I2.FOO))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`INPUT_1_FOO` INTEGER KEY, `INPUT_1_BAR` INTEGER, `I2_BAR` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "INPUT_1", "INPUT_2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS INPUT_1_BAR", "ROWTIME AS INPUT_1_ROWTIME", "FOO AS INPUT_1_FOO" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` INTEGER KEY, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS I2_BAR", "ROWTIME AS I2_ROWTIME", "FOO AS I2_FOO" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000, + "keyColName" : "INPUT_1_FOO" + }, + "selectExpressions" : [ "INPUT_1_BAR AS INPUT_1_BAR", "I2_BAR AS I2_BAR" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/spec.json new file mode 100644 index 000000000000..7de251918d82 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/spec.json @@ -0,0 +1,108 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589386994372, + "path" : "query-validation-tests/identifiers.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "unaliased left aliased right", + "inputs" : [ { + "topic" : "t1", + "key" : 1, + "value" : { + "bar" : 2 + } + }, { + "topic" : "t2", + "key" : 1, + "value" : { + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "INPUT_1_BAR" : 2, + "I2_BAR" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "t2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT_1 (FOO INT KEY, bar INT) WITH (kafka_topic='t1', value_format='JSON');", "CREATE STREAM INPUT_2 (FOO INT KEY, bar INT) WITH (kafka_topic='t2', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT INPUT_1.FOO, INPUT_1.BAR, I2.BAR FROM INPUT_1 JOIN INPUT_2 AS I2 WITHIN 1 MINUTE ON INPUT_1.FOO = I2.FOO;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "t1", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "t2", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/topology new file mode 100644 index 000000000000..b39e415a7480 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/identifiers_-_unaliased_left_aliased_right/6.0.0_1589386994372/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [t2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/plan.json new file mode 100644 index 000000000000..c2aabdeefd06 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/plan.json @@ -0,0 +1,131 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='delimited');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` BIGINT KEY, `ID` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM TEST TEST\nPARTITION BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `K` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`K` BIGINT KEY, `ID` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "K AS K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/spec.json new file mode 100644 index 000000000000..7b9572a9e048 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/spec.json @@ -0,0 +1,67 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013555, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "bigint key field", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : "0" + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) with (kafka_topic='test_topic', value_format = 'delimited');", "CREATE STREAM OUTPUT AS select * from TEST partition by ID;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "ID BIGINT KEY, K BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/topology new file mode 100644 index 000000000000..63f0d28f9a1f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_bigint_key_field/6.0.0_1589387013555/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PartitionBy-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/plan.json new file mode 100644 index 000000000000..e777830d2d3a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/plan.json @@ -0,0 +1,204 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`K` STRING KEY, `A` STRING, `B` STRING", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`K` STRING KEY, `C` STRING, `D` STRING", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.A A,\n L.K L_K,\n R.K R_K\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.K = R.K))\nPARTITION BY L.A\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`A` STRING KEY, `L_K` STRING, `R_K` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `A` STRING, `B` STRING" + }, + "selectExpressions" : [ "A AS L_A", "B AS L_B", "ROWTIME AS L_ROWTIME", "K AS L_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `C` STRING, `D` STRING" + }, + "selectExpressions" : [ "C AS R_C", "D AS R_D", "ROWTIME AS R_ROWTIME", "K AS R_K" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_K" + }, + "keyExpression" : "L_A" + }, + "selectExpressions" : [ "L_K AS L_K", "R_K AS R_K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/spec.json new file mode 100644 index 000000000000..9082b76eb14f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/spec.json @@ -0,0 +1,115 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013813, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by Key in join on Key ALIASED", + "inputs" : [ { + "topic" : "LEFT", + "key" : "join", + "value" : { + "A" : "join", + "B" : "b" + } + }, { + "topic" : "RIGHT", + "key" : "join", + "value" : { + "C" : "join", + "D" : "d" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "join", + "value" : { + "L_K" : "join", + "R_K" : "join" + } + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT L.A, L.K, R.K FROM L JOIN R WITHIN 10 SECONDS ON L.K = R.K PARTITION BY L.A;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream" + } ], + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/topology new file mode 100644 index 000000000000..cb84d748371c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_Key_in_join_on_Key_ALIASED/6.0.0_1589387013813/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> PartitionBy-SelectKey + <-- Join-this-join, Join-other-join + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- Join-merge + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000013 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000013 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/plan.json new file mode 100644 index 000000000000..6d7c5e550c25 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/plan.json @@ -0,0 +1,197 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`K` STRING KEY, `A` STRING, `B` STRING", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`K` STRING KEY, `C` STRING, `D` STRING", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.K LK,\n R.K R_K\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.K = R.K))\nPARTITION BY L.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`LK` STRING KEY, `R_K` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `A` STRING, `B` STRING" + }, + "selectExpressions" : [ "A AS L_A", "B AS L_B", "ROWTIME AS L_ROWTIME", "K AS L_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `C` STRING, `D` STRING" + }, + "selectExpressions" : [ "C AS R_C", "D AS R_D", "ROWTIME AS R_ROWTIME", "K AS R_K" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_K" + }, + "selectExpressions" : [ "R_K AS R_K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/spec.json new file mode 100644 index 000000000000..caef8cea3625 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013690, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by key in join on key", + "inputs" : [ { + "topic" : "LEFT", + "key" : "join", + "value" : { + "A" : "join", + "B" : "b" + } + }, { + "topic" : "RIGHT", + "key" : "join", + "value" : { + "C" : "join", + "D" : "d" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "join", + "value" : { + "R_K" : "join" + } + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT L.K AS LK, R.K FROM L JOIN R WITHIN 10 SECONDS ON L.K = R.K PARTITION BY L.K;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/topology new file mode 100644 index 000000000000..f3523932dc41 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_key/6.0.0_1589387013690/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/plan.json new file mode 100644 index 000000000000..aa3b549541e5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/plan.json @@ -0,0 +1,218 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`K` STRING KEY, `A` STRING, `B` STRING", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`K` STRING KEY, `C` STRING, `D` STRING", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.K LK,\n R.K R_K\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.B = R.D))\nPARTITION BY L.K\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`LK` STRING KEY, `R_K` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `A` STRING, `B` STRING" + }, + "keyExpression" : "B" + }, + "selectExpressions" : [ "A AS L_A", "B AS L_B", "ROWTIME AS L_ROWTIME", "K AS L_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `C` STRING, `D` STRING" + }, + "keyExpression" : "D" + }, + "selectExpressions" : [ "C AS R_C", "D AS R_D", "ROWTIME AS R_ROWTIME", "K AS R_K" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_B" + }, + "keyExpression" : "L_K" + }, + "selectExpressions" : [ "R_K AS R_K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/spec.json new file mode 100644 index 000000000000..01fc6b9f6d78 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/spec.json @@ -0,0 +1,129 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013749, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by key in join on non-key", + "inputs" : [ { + "topic" : "LEFT", + "key" : "a", + "value" : { + "A" : "a", + "B" : "join" + } + }, { + "topic" : "RIGHT", + "key" : "c", + "value" : { + "C" : "c", + "D" : "join" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : { + "R_K" : "c" + } + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT L.K AS LK, R.K FROM L JOIN R WITHIN 10 SECONDS ON L.B = R.D PARTITION BY L.K;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/topology new file mode 100644 index 000000000000..18b4057c0a24 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_key_in_join_on_non-key/6.0.0_1589387013749/topology @@ -0,0 +1,66 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> PartitionBy-SelectKey + <-- Join-this-join, Join-other-join + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- Join-merge + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000021 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000021 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/plan.json new file mode 100644 index 000000000000..54d80ab22ab5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/plan.json @@ -0,0 +1,218 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`K` STRING KEY, `A` STRING, `B` STRING", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`K` STRING KEY, `C` STRING, `D` STRING", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.B B,\n L.K L_K,\n R.K R_K\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.A = R.C))\nPARTITION BY L.B\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`B` STRING KEY, `L_K` STRING, `R_K` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `A` STRING, `B` STRING" + }, + "keyExpression" : "A" + }, + "selectExpressions" : [ "A AS L_A", "B AS L_B", "ROWTIME AS L_ROWTIME", "K AS L_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `C` STRING, `D` STRING" + }, + "keyExpression" : "C" + }, + "selectExpressions" : [ "C AS R_C", "D AS R_D", "ROWTIME AS R_ROWTIME", "K AS R_K" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_A" + }, + "keyExpression" : "L_B" + }, + "selectExpressions" : [ "L_K AS L_K", "R_K AS R_K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/spec.json new file mode 100644 index 000000000000..495d00cabc90 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/spec.json @@ -0,0 +1,134 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013877, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by non-Key in join on Key", + "inputs" : [ { + "topic" : "LEFT", + "key" : "join", + "value" : { + "A" : "join", + "B" : "b" + } + }, { + "topic" : "RIGHT", + "key" : "join", + "value" : { + "C" : "join", + "D" : "d" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "b", + "value" : { + "L_K" : "join", + "R_K" : "join" + } + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT L.B, L.K, R.K FROM L JOIN R WITHIN 10 SECONDS ON L.A = R.C PARTITION BY L.B;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream" + } ], + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/topology new file mode 100644 index 000000000000..18b4057c0a24 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_Key/6.0.0_1589387013877/topology @@ -0,0 +1,66 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> PartitionBy-SelectKey + <-- Join-this-join, Join-other-join + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- Join-merge + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000021 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000021 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/plan.json new file mode 100644 index 000000000000..75ce3cb02587 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`K` STRING KEY, `A` STRING, `B` STRING", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`K` STRING KEY, `C` STRING, `D` STRING", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.B B,\n L.K L_K,\n R.K R_K\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.B = R.D))\nPARTITION BY L.B\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`B` STRING KEY, `L_K` STRING, `R_K` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `A` STRING, `B` STRING" + }, + "keyExpression" : "B" + }, + "selectExpressions" : [ "A AS L_A", "B AS L_B", "ROWTIME AS L_ROWTIME", "K AS L_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `C` STRING, `D` STRING" + }, + "keyExpression" : "D" + }, + "selectExpressions" : [ "C AS R_C", "D AS R_D", "ROWTIME AS R_ROWTIME", "K AS R_K" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_B" + }, + "selectExpressions" : [ "L_K AS L_K", "R_K AS R_K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/spec.json new file mode 100644 index 000000000000..a041c3e81081 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/spec.json @@ -0,0 +1,134 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013948, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by non-Key in join on non-Key", + "inputs" : [ { + "topic" : "LEFT", + "key" : "a", + "value" : { + "A" : "a", + "B" : "join" + } + }, { + "topic" : "RIGHT", + "key" : "c", + "value" : { + "C" : "c", + "D" : "join" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "join", + "value" : { + "L_K" : "a", + "R_K" : "c" + } + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (K STRING KEY, A STRING, B STRING) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (K STRING KEY, C STRING, D STRING) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT L.B, L.K, R.K FROM L JOIN R WITHIN 10 SECONDS ON L.B = R.D PARTITION BY L.B;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream" + } ], + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/topology new file mode 100644 index 000000000000..0eee5fb318ac --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_non-Key_in_join_on_non-Key/6.0.0_1589387013948/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/plan.json new file mode 100644 index 000000000000..2b613e94549b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/plan.json @@ -0,0 +1,131 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='delimited');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM REPARTITIONED AS SELECT *\nFROM TEST TEST\nPARTITION BY TEST.NAME\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "REPARTITIONED", + "schema" : "`NAME` STRING KEY, `ID` BIGINT, `VALUE` BIGINT, `K` BIGINT", + "topicName" : "REPARTITIONED", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "REPARTITIONED", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "REPARTITIONED" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "sourceSchema" : "`K` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "NAME" + }, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "K AS K" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED" + } + }, + "topicName" : "REPARTITIONED" + }, + "queryId" : "CSAS_REPARTITIONED_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/spec.json new file mode 100644 index 000000000000..41a6cdf641a1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/spec.json @@ -0,0 +1,67 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387013598, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_REPARTITIONED_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_REPARTITIONED_0.REPARTITIONED" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by with null value", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : null + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,50" + } ], + "outputs" : [ { + "topic" : "REPARTITIONED", + "key" : null, + "value" : null + }, { + "topic" : "REPARTITIONED", + "key" : "zero", + "value" : "0,50,0" + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "REPARTITIONED", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K BIGINT KEY, ID bigint, NAME varchar, VALUE bigint) with (kafka_topic='test_topic', value_format = 'delimited');", "CREATE STREAM REPARTITIONED AS select * from TEST partition by name;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "REPARTITIONED", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "DELIMITED" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/topology new file mode 100644 index 000000000000..ec9879fdad67 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_with_null_value/6.0.0_1589387013598/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PartitionBy-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000004 (topic: REPARTITIONED) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/plan.json new file mode 100644 index 000000000000..a929ff7057fb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/plan.json @@ -0,0 +1,194 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (`the key` STRING KEY, `SELECT` STRING, `field!` STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`the key` STRING KEY, `SELECT` STRING, `field!` STRING", + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE R (`with.dot` STRING PRIMARY KEY, `field 0` STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "R", + "schema" : "`with.dot` STRING KEY, `field 0` STRING", + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM JOINED AS SELECT *\nFROM L L\nLEFT OUTER JOIN R R ON ((L.`SELECT` = R.`with.dot`))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "JOINED", + "schema" : "`L_SELECT` STRING KEY, `L_the key` STRING, `L_field!` STRING, `R_with.dot` STRING, `R_field 0` STRING", + "topicName" : "JOINED", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "JOINED", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "JOINED" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`the key` STRING KEY, `SELECT` STRING, `field!` STRING" + }, + "keyExpression" : "`SELECT`" + }, + "selectExpressions" : [ "`SELECT` AS L_SELECT", "`field!` AS `L_field!`", "ROWTIME AS L_ROWTIME", "`the key` AS `L_the key`" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`with.dot` STRING KEY, `field 0` STRING" + }, + "selectExpressions" : [ "`field 0` AS `R_field 0`", "ROWTIME AS R_ROWTIME", "`with.dot` AS `R_with.dot`" ] + }, + "keyColName" : "L_SELECT" + }, + "selectExpressions" : [ "`L_the key` AS `L_the key`", "`L_field!` AS `L_field!`", "`R_with.dot` AS `R_with.dot`", "`R_field 0` AS `R_field 0`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "JOINED" + }, + "queryId" : "CSAS_JOINED_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/spec.json new file mode 100644 index 000000000000..5c66d9c2414b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_joins_using_fields_that_require_quotes/6.0.0_1589387014881/spec.json @@ -0,0 +1,106 @@ +{ + "version" : "6.0.0", + "timestamp" : 1589387014881, + "path" : "query-validation-tests/quoted-identifiers.json", + "schemas" : { + "CSAS_JOINED_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_JOINED_0.KafkaTopic_Left.Source" : "STRUCT