From 5e9cb562af32fdc94df8c6ff08e93306b171f34c Mon Sep 17 00:00:00 2001 From: Andy Coates Date: Mon, 2 Mar 2020 17:18:32 +0000 Subject: [PATCH] feat: remove restriction that key columns must be called ROWKEY Fixes: https://github.com/confluentinc/ksql/issues/3536 --- .../concepts/collections/inserting-events.md | 2 +- docs-md/concepts/events.md | 12 +- docs-md/concepts/queries/pull.md | 11 +- docs-md/concepts/schemas.md | 15 +- docs-md/concepts/stream-processing.md | 5 +- .../developer-guide/joins/partition-data.md | 109 ++++++------ .../ksqldb-reference/create-stream.md | 11 +- .../ksqldb-reference/create-table.md | 16 +- .../ksqldb-reference/insert-values.md | 43 +++-- .../ksqldb-reference/select-pull-query.md | 6 +- .../ksqldb-reference/select-push-query.md | 4 +- docs-md/developer-guide/syntax-reference.md | 2 +- .../test-and-debug/ksqldb-testing-tool.md | 26 +-- .../installation/server-config/avro-schema.md | 17 +- docs-md/tutorials/embedded-connect.md | 16 +- docs-md/tutorials/examples.md | 32 ++-- .../confluent/ksql/cli/console/Console.java | 6 +- .../ksql/schema/ksql/LogicalSchema.java | 16 +- .../ksql/schema/ksql/LogicalSchemaTest.java | 10 +- .../io/confluent/ksql/analyzer/Analysis.java | 10 ++ .../io/confluent/ksql/analyzer/Analyzer.java | 20 ++- .../ksql/analyzer/PullQueryValidator.java | 4 +- .../ddl/commands/CreateSourceFactory.java | 7 +- .../ksql/engine/InsertValuesExecutor.java | 60 ++++--- .../ksql/planner/LogicalPlanner.java | 6 + .../ksql/planner/plan/AggregateNode.java | 1 + .../confluent/ksql/query/QueryExecutor.java | 5 + .../ksql/structured/SchemaKStream.java | 11 +- .../ksql/structured/SchemaKTable.java | 5 +- .../ksql/analyzer/SourceSchemasTest.java | 42 +++-- .../ksql/codegen/CodeGenRunnerTest.java | 3 +- .../ddl/commands/CreateSourceFactoryTest.java | 15 +- .../ksql/ddl/commands/DdlCommandExecTest.java | 3 +- .../ksql/engine/InsertValuesExecutorTest.java | 124 +++++++------- .../ks/KsMaterializationFunctionalTest.java | 1 + .../physical/PhysicalPlanBuilderTest.java | 4 +- .../ksql/planner/plan/DataSourceNodeTest.java | 6 +- .../ksql/planner/plan/JoinNodeTest.java | 45 ++--- .../confluent/ksql/util/ItemDataProvider.java | 1 + .../confluent/ksql/datagen/RowGenerator.java | 4 +- .../ddl/commands/CreateSourceCommand.java | 11 +- .../expression/tree/ColumnReferenceExp.java | 1 + .../ksql/execution/util/StructKeyUtil.java | 21 ++- .../ddl/commands/CreateSourceCommandTest.java | 14 +- .../transform/select/SelectionTest.java | 13 +- .../execution/util/StructKeyUtilTest.java | 9 +- ksql-functional-tests/README.md | 6 +- .../6.0.0_1583166643536/plan.json | 145 ++++++++++++++++ .../6.0.0_1583166643536/spec.json | 23 +++ .../6.0.0_1583166643536/topology | 13 ++ .../6.0.0_1583166643478/plan.json | 145 ++++++++++++++++ .../6.0.0_1583166643478/spec.json | 23 +++ .../6.0.0_1583166643478/topology | 13 ++ .../query-validation-tests/elements.json | 39 ++++- .../query-validation-tests/joins.json | 4 +- .../query-validation-tests/key-field.json | 2 +- .../query-validation-tests/key-schemas.json | 42 ++++- .../system-columns.json | 12 +- .../insert-values.json | 78 +++++---- ...eries-against-materialized-aggregates.json | 32 +++- .../metastore/model/StructuredDataSource.java | 11 ++ .../model/StructuredDataSourceTest.java | 5 +- .../confluent/ksql/util/MetaStoreFixture.java | 1 + .../ksql/parser/tree/TableElements.java | 21 ++- .../confluent/ksql/parser/KsqlParserTest.java | 3 +- .../ksql/parser/SchemaParserTest.java | 17 +- .../ksql/parser/SqlFormatterTest.java | 15 +- .../ksql/parser/tree/TableElementTest.java | 4 +- .../ksql/parser/tree/TableElementsTest.java | 20 +++ .../server/execution/PullQueryExecutor.java | 155 ++++++++++-------- .../api/integration/ApiIntegrationTest.java | 2 +- .../confluent/ksql/serde/GenericRowSerDe.java | 2 +- .../streams/GroupByParamsFactory.java | 26 +-- .../ksql/execution/streams/SourceBuilder.java | 28 ++-- .../execution/streams/StepSchemaResolver.java | 1 + .../streams/JoinParamsFactoryTest.java | 5 +- .../execution/streams/SourceBuilderTest.java | 9 +- .../streams/StepSchemaResolverTest.java | 12 +- .../streams/StreamGroupByBuilderTest.java | 4 +- .../streams/StreamSelectKeyBuilderTest.java | 6 +- .../streams/TableGroupByBuilderTest.java | 5 +- .../KsqlMaterializationTest.java | 3 +- 82 files changed, 1170 insertions(+), 567 deletions(-) create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology diff --git a/docs-md/concepts/collections/inserting-events.md b/docs-md/concepts/collections/inserting-events.md index 360b46cc04cc..4818b35d1e9e 100644 --- a/docs-md/concepts/collections/inserting-events.md +++ b/docs-md/concepts/collections/inserting-events.md @@ -34,7 +34,7 @@ INSERT INTO all_publications (author, title) VALUES ('C.S. Lewis', 'The Silver C Any column that doesn't get a value explicitly 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 +schema, with the key columns first. If columns are specified, the order doesn’t matter. You can specify `ROWTIME` as an explicit column, but it’s not required when you omit the column specifications. diff --git a/docs-md/concepts/events.md b/docs-md/concepts/events.md index 8236cc4828c5..ae0040e48248 100644 --- a/docs-md/concepts/events.md +++ b/docs-md/concepts/events.md @@ -47,10 +47,12 @@ to {{ site.ak }}, they also describe the time at which the event was true. ksqlDB aims to raise the abstraction from working with a lower-level stream processor. Usually, an event is called a "row", as if it were a row in a -relational database. Each row is composed of a series of columns. Most columns -represent fields in the value of an event, but there are a few extra columns. -In particular, there are the `ROWKEY` and `ROWTIME` columns that represent the -key and time of the event. These system columns are present on every row. -In addition, windowed sources have `WINDOWSTART` and `WINDOWEND` columns. +relational database. Each row is composed of a series of columns. +ksqlDB currently supports only a single key column and multiple value columns. +These columns represent fields in the key and value of an event, respectively. +There are also a few extra system columns. The `ROWTIME` column that represent the +time of the event, in milliseconds. This system column is present on every row. +In addition, windowed sources have `WINDOWSTART` and `WINDOWEND` columns, which +represent the bounds on the window, in milliseconds. Page last revised on: {{ git_revision_date }} \ No newline at end of file diff --git a/docs-md/concepts/queries/pull.md b/docs-md/concepts/queries/pull.md index 4099f3e7c6dd..435cfbf7a40f 100644 --- a/docs-md/concepts/queries/pull.md +++ b/docs-md/concepts/queries/pull.md @@ -34,10 +34,9 @@ Pull query features and limitations - Pull queries are currently available only against materialized aggregate tables, which means tables that are created by using a persistent query with a GROUP BY clause. -- For non-windowed aggregations, pull queries only support looking up events - by key. -- WHERE clauses can only have `ROWKEY=x`-style bounds for non-windowed tables. -- Windowed tables support bounds on WINDOWSTART using operators +- For non-windowed tables, pull queries only support looking up events + by key, i.e. the WHERE clause can only have `=x`-style filters. +- Windowed tables support additional bounds on WINDOWSTART using operators `<=`, `<`, `=`, `>`, `>=`. - JOIN, PARTITION BY, GROUP BY and WINDOW clauses aren't supported. - SELECT statements can contain column arithmetic and function calls. @@ -48,12 +47,12 @@ Example pull query The following pull query gets all events for the specified user that have a timestamp within the specified time window. The WHERE clause must contain a -single value of `ROWKEY` to retrieve and may optionally include bounds on +single value of the key, `userId` in this case, to retrieve and may optionally include bounds on WINDOWSTART if the materialized table is windowed. ```sql SELECT * FROM user_location - WHERE ROWKEY = 'user19r7t33' + WHERE userId = 'user19r7t33' AND '2019-10-02T21:31:16' <= WINDOWSTART AND WINDOWSTART <= '2019-10-03T21:31:16'; ``` diff --git a/docs-md/concepts/schemas.md b/docs-md/concepts/schemas.md index 0c92a3bdb7bb..39f8cb7778be 100644 --- a/docs-md/concepts/schemas.md +++ b/docs-md/concepts/schemas.md @@ -17,18 +17,17 @@ underlying {{ site.ak }} topic. A column is defined by a combination of its [name](#valid-identifiers), its [SQL data type](#sql-data-types), and possibly a namespace. -Key columns have a `KEY` namespace suffix. Key columns have the following restrictions: - * The can only be a single key column, currently. - * The key column must be named `ROWKEY` in the KSQL schema. +Key columns have a `KEY` namespace suffix. The can only be a single key column, currently, and the +key column can have any [valid name](#valid-identifiers). -Value columns have no namespace suffix. There can be one or more value columns amd the value columns -can have any name. +Value columns have no namespace suffix. There can be one or more value columns and the value columns +can have any [valid name](#valid-identifiers). -For example, the following declares a schema with a single `INT` key column and several value -columns: +For example, the following declares a schema with a single `INT` key column, containing some Id, and +several value columns: ```sql -ROWKEY INT KEY, ID BIGINT, STRING NAME, ADDRESS ADDRESS_TYPE +ID INT KEY, STRING NAME, ADDRESS ADDRESS_TYPE ``` ## Valid Identifiers diff --git a/docs-md/concepts/stream-processing.md b/docs-md/concepts/stream-processing.md index b7a180202550..b7dd06fa4ad6 100644 --- a/docs-md/concepts/stream-processing.md +++ b/docs-md/concepts/stream-processing.md @@ -27,9 +27,10 @@ collection by using the `SELECT` statement on an existing collection. The result of the inner `SELECT` feeds into the outer declared collection. You don't need to declare a schema when deriving a new collection, because ksqlDB infers the column names and types from the inner `SELECT` statement. The -`ROWKEY` of the row remains the same, unless the query includes either a +key columns of the row remains the same, unless the query includes either a `PARTITION BY` or `GROUP BY` clause. The value of the `ROWTIME` column -sets the timestamp of the record written to {{ site.ak }}. The value of system columns +sets the timestamp of the record written to {{ site.ak }}, unless overridden by +specifying a different column in the WITH clause. The value of system columns can not be set in the `SELECT`. Here are a few examples of deriving between the different collection types. diff --git a/docs-md/developer-guide/joins/partition-data.md b/docs-md/developer-guide/joins/partition-data.md index a0b842f538ea..8b470d1654ea 100644 --- a/docs-md/developer-guide/joins/partition-data.md +++ b/docs-md/developer-guide/joins/partition-data.md @@ -32,31 +32,46 @@ ksqlDB requires keys to use the `KAFKA` format. For more information, see 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`. - -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 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). +important to understand how keys are defined. + +When you create a table or stream by using a CREATE TABLE or CREATE STREAM statement, the key of the +table or stream should be the same as that of the records in the underlying {{ site.ak }} topic. +You can set the type of the key column in the statement. Key columns are identified by the `KEY` +namespace. For example, the following statement creates a stream with a key column named `ID` of +type `BIGINT`: + + +```sql +CREATE STREAM USERS (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='users', value_format='json'); +``` + +When you create a stream by using a CREATE STREAM AS SELECT statement, the key of the resulting +stream is determined as follows: + +- If the statement includes a PARTITION BY clause the, the key type is set to the type of the + PARTITION BY clause. The key column will have the default name of `ROWKEY`. +- If the statement does not include a PARTITION BY clause, the key is copied over from the key of + the first stream in the FROM clause. + +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. The key column + will have the default name of `ROWKEY` and the grouping column(s) determine the type of the key + of the resulting table. + - When grouping by a single column or expression, the type of the key column in the + resulting table matches the type of this column or expression. + - When grouping by multiple columns or expressions, the type of the key column in the + resulting table 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 first table in the FROM clause. +- If the FROM clause contains only tables and has a GROUP BY clause, The key column will have the + default name of `ROWKEY` and the grouping column(s) determine the type of the key of the resulting + table. + - When grouping by a single column or expression, the type of the key column in the + resulting table matches the type of this column or expression. + - When grouping by multiple columns or expressions, the type of the key column in the + resulting table is an [SQL `STRING`](../../concepts/schemas). The following example shows a `users` table joined with a `clicks` stream on the `userId` column. The `users` table has the correct primary key @@ -65,21 +80,17 @@ doesn't have a defined key, and ksqlDB must repartition it on the joining column (`userId`) and assign the key before performing the join. ```sql - -- clicks stream, with an unknown key. - -- the schema of stream clicks is: ROWTIME BIGINT | ROWKEY STRING | USERID BIGINT | URL STRING + -- clicks stream, with an unknown key: + -- the schema of stream clicks is: 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: ROWTIME BIGINT | ROWKEY BIGINT | USERID BIGINT | FULLNAME STRING - CREATE TABLE users (ROWKEY BIGINT 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 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: + -- the schema of the result is: USERID BIGINT KEY | CLICKS_URL STRING | USERS_FULLNAME STRING 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 @@ -112,29 +123,27 @@ Records with the exact same user id on both sides will be joined. If the schema of the columns you wish to join on don't match, it may be possible to `CAST` one side to match the other. For example, if one side of the join had a `INT` userId column, and the other a `LONG`, then you may choose to cast -the `INT` side to a `LONG`: +the `INT` side to a `LONG`. Note, this will incur the cost of an internal repartition of the data: ```sql -- stream with INT userId CREATE STREAM clicks (userId INT, url STRING) WITH(kafka_topic='clickstream', value_format='json'); -- table with BIGINT userId stored in they key: - CREATE TABLE users (ROWKEY BIGINT KEY, fullName STRING) WITH(kafka_topic='users', value_format='json'); + CREATE TABLE users (userId BIGINT 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.userId; ``` +Tables or streams created on top of existing Kafka topics, for example those created with +a `CREATE TABLE` or `CREATE STREAM` statements, are keyed on the data held in the key of the records +in the Kafka topic. ksqlDB expects the data to be in the `KAFKA` format. -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 -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 -is keyed on. +Tables or streams created inside ksqlDB from other sources, for example those created with +a `CREATE TABLE AS SELECT` or `CREATE STREAM AS SELECT` statements, will copy the key from their +source(s) unless there is an explicit `GROUP BY` or `PARTITION BY` clauses, which can change what +they are keyed on. See the [Keys](#keys) section above for more information. !!! note ksqlDB automatically repartitions a stream if a join requires it, but ksqlDB @@ -145,7 +154,7 @@ is keyed on. If you are using the same sources in more than one join that requires the data to be repartitioned, you may prefer to repartition manually to avoid ksqlDB -repartitioning multiple times. +repartitioning the same data multiple times. To repartition a stream, use the PARTITION BY clause. Be aware that Kafka guarantees the relative order of any two messages from one source partition @@ -182,10 +191,10 @@ the partition counts of the source topics, or repartition one side to match the partition count of the other. The following example creates a repartitioned stream, maintaining the existing -key, with the specified number of partitions. +key, `productId` in this case, with the specified number of partitions. ```sql -CREATE STREAM products_rekeyed WITH (PARTITIONS=6) AS SELECT * FROM products PARTITION BY ROWKEY; +CREATE STREAM products_rekeyed WITH (PARTITIONS=6) AS SELECT * FROM products PARTITION BY productId; ``` ### Records Have the Same Partitioning Strategy diff --git a/docs-md/developer-guide/ksqldb-reference/create-stream.md b/docs-md/developer-guide/ksqldb-reference/create-stream.md index 8b0f17dc1b0f..38e23c9f54b0 100644 --- a/docs-md/developer-guide/ksqldb-reference/create-stream.md +++ b/docs-md/developer-guide/ksqldb-reference/create-stream.md @@ -24,9 +24,11 @@ Create a new stream with the specified columns and properties. Columns can be any of the [data types](../syntax-reference.md#ksqldb-data-types) supported by ksqlDB. -ksqlDB adds the implicit columns `ROWTIME` and `ROWKEY` to every stream -and table, which represent the corresponding Kafka message timestamp and -message key, respectively. The timestamp has milliseconds accuracy. +To maintain backwards compatibility with older versions, ksqlDB adds an implicit `ROWKEY STRING KEY` +column if no explicit key column is provided. + +ksqlDB adds the implicit column `ROWTIME` to every stream, which represent the corresponding Kafka +message timestamp. The timestamp has milliseconds accuracy. The WITH clause supports the following properties: @@ -37,7 +39,6 @@ The WITH clause supports the following properties: | PARTITIONS | The number of partitions in the backing topic. This property must be set if creating a STREAM without an existing topic (the command will fail if the topic does not exist). | | REPLICAS | The number of replicas in the backing topic. If this property is not set but PARTITIONS is set, then the default Kafka cluster configuration for replicas will be used for creating a new topic. | | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not an actual space or tab character. | -| KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka message value, you may set this property to associate the corresponding field/column with the implicit `ROWKEY` column (message key). If set, ksqlDB uses it as an optimization hint to determine if repartitioning can be avoided when performing aggregations and joins. You can only use this if the key format in Kafka is `VARCHAR` or `STRING`. Do not use this hint if the message key format in Kafka is `AVRO` or `JSON`. See [Key Requirements](../syntax-reference.md#key-requirements) for more information. | | TIMESTAMP | By default, the implicit `ROWTIME` column is the timestamp of the message in the Kafka topic. The TIMESTAMP property can be used to override `ROWTIME` with the contents of the specified field/column within the Kafka message value (similar to timestamp extractors in Kafka's Streams API). Timestamps have a millisecond accuracy. Time-based operations, such as windowing, will process a record according to the timestamp in `ROWTIME`. | | TIMESTAMP_FORMAT | Used in conjunction with TIMESTAMP. If not set will assume that the timestamp field is a `bigint`. If it is set, then the TIMESTAMP field must be of type `varchar` and have a format that can be parsed with the java `DateTimeFormatter`. If your timestamp format has characters requiring single quotes, you can escape them with successive single quotes, `''`, for example: `'yyyy-MM-dd''T''HH:mm:ssX'`. For more information on timestamp formats, see [DateTimeFormatter](https://cnfl.io/java-dtf). | | WRAP_SINGLE_VALUE | Controls how values are deserialized where the value schema contains only a single field. The setting controls how ksqlDB will deserialize the value of the records in the supplied `KAFKA_TOPIC` that contain only a single field.
If set to `true`, ksqlDB expects the field to have been serialized as a named field within a record.
If set to `false`, ksqlDB expects the field to have been serialized as an anonymous value.
If not supplied, the system default, defined by [ksql.persistence.wrap.single.values](../../operate-and-deploy/installation/server-config/config-reference.md#ksqlpersistencewrapsinglevalues) and defaulting to `true`, is used.
**Note:** `null` values have special meaning in ksqlDB. Care should be taken when dealing with single-field schemas where the value can be `null`. For more information, see [Single field (un)wrapping](../serialization.md#single-field-unwrapping).
**Note:** Supplying this property for formats that do not support wrapping, for example `DELIMITED`, or when the value schema has multiple fields, will result in an error. | @@ -58,7 +59,7 @@ Example ------- ```sql -CREATE STREAM pageviews (viewtime BIGINT, user_id VARCHAR, page_id VARCHAR) +CREATE STREAM pageviews (userId BIGINT KEY, viewtime BIGINT, page_id VARCHAR) WITH (VALUE_FORMAT = 'JSON', KAFKA_TOPIC = 'my-pageviews-topic'); ``` diff --git a/docs-md/developer-guide/ksqldb-reference/create-table.md b/docs-md/developer-guide/ksqldb-reference/create-table.md index d7cbdbb64685..08685d302bba 100644 --- a/docs-md/developer-guide/ksqldb-reference/create-table.md +++ b/docs-md/developer-guide/ksqldb-reference/create-table.md @@ -21,13 +21,11 @@ Create a new table with the specified columns and properties. Columns can be any of the [data types](../syntax-reference.md#ksqldb-data-types) supported by ksqlDB. -ksqlDB adds the implicit columns `ROWTIME` and `ROWKEY` to every stream -and table, which represent the corresponding Kafka message timestamp and -message key, respectively. The timestamp has milliseconds accuracy. +To maintain backwards compatibility with older versions, ksqlDB adds an implicit `ROWKEY STRING KEY` +column if no explicit key column is provided. -When creating a table from a Kafka topic, ksqlDB requries the message key -to be a `VARCHAR` aka `STRING`. If the message key is not of this type -follow the instructions in [Key Requirements](../syntax-reference.md#key-requirements). +ksqlDB adds the implicit column `ROWTIME` to every table, which represent the corresponding Kafka +message timestamp. The timestamp has milliseconds accuracy. The WITH clause supports the following properties: @@ -38,7 +36,6 @@ The WITH clause supports the following properties: | PARTITIONS | The number of partitions in the backing topic. This property must be set if creating a TABLE without an existing topic (the command will fail if the topic does not exist). | | REPLICAS | The number of replicas in the backing topic. If this property is not set but PARTITIONS is set, then the default Kafka cluster configuration for replicas will be used for creating a new topic. | | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not an actual space or tab character. | -| KEY | **Optimization hint:** If the Kafka message key is also present as a field/column in the Kafka message value, you may set this property to associate the corresponding field/column with the implicit `ROWKEY` column (message key). If set, ksqlDB uses it as an optimization hint to determine if repartitioning can be avoided when performing aggregations and joins. You can only use this if the key format in kafka is `VARCHAR` or `STRING`. Do not use this hint if the message key format in kafka is `AVRO` or `JSON`. For more information, see [Key Requirements](../syntax-reference.md#key-requirements). | | TIMESTAMP | By default, the implicit `ROWTIME` column is the timestamp of the message in the Kafka topic. The TIMESTAMP property can be used to override `ROWTIME` with the contents of the specified field/column within the Kafka message value (similar to timestamp extractors in the Kafka Streams API). Timestamps have a millisecond accuracy. Time-based operations, such as windowing, will process a record according to the timestamp in `ROWTIME`. | | TIMESTAMP_FORMAT | Used in conjunction with TIMESTAMP. If not set will assume that the timestamp field is a `bigint`. If it is set, then the TIMESTAMP field must be of type varchar and have a format that can be parsed with the Java `DateTimeFormatter`. If your timestamp format has characters requiring single quotes, you can escape them with two successive single quotes, `''`, for example: `'yyyy-MM-dd''T''HH:mm:ssX'`. For more information on timestamp formats, see [DateTimeFormatter](https://cnfl.io/java-dtf). | | WRAP_SINGLE_VALUE | Controls how values are deserialized where the values schema contains only a single field. The setting controls how ksqlDB will deserialize the value of the records in the supplied `KAFKA_TOPIC` that contain only a single field.
If set to `true`, ksqlDB expects the field to have been serialized as named field within a record.
If set to `false`, ksqlDB expects the field to have been serialized as an anonymous value.
If not supplied, the system default, defined by [ksql.persistence.wrap.single.values](../../operate-and-deploy/installation/server-config/config-reference.md#ksqlpersistencewrapsinglevalues) and defaulting to `true`, is used.
**Note:** `null` values have special meaning in ksqlDB. Care should be taken when dealing with single-field schemas where the value can be `null`. For more information, see [Single field (un)wrapping](../serialization.md#single-field-unwrapping).
**Note:** Supplying this property for formats that do not support wrapping, for example `DELIMITED`, or when the value schema has multiple fields, will result in an error. | @@ -55,9 +52,8 @@ Example ------- ```sql -CREATE TABLE users (usertimestamp BIGINT, user_id VARCHAR, gender VARCHAR, region_id VARCHAR) WITH ( - KAFKA_TOPIC = 'my-users-topic', - KEY = 'user_id'); +CREATE TABLE users (userId BIGINT, usertimestamp BIGINT, gender VARCHAR, region_id VARCHAR) WITH ( + KAFKA_TOPIC = 'my-users-topic'); ``` Page last revised on: {{ git_revision_date }} diff --git a/docs-md/developer-guide/ksqldb-reference/insert-values.md b/docs-md/developer-guide/ksqldb-reference/insert-values.md index e2ed10cfd7ef..277c57a5df8a 100644 --- a/docs-md/developer-guide/ksqldb-reference/insert-values.md +++ b/docs-md/developer-guide/ksqldb-reference/insert-values.md @@ -21,44 +21,41 @@ 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 no columns are specified, a value for every column is expected in the same order as +the schema. If columns are specified, the order of values must match the order of the names. +Any column not explicitly given a value is set to `null`. !!! note `ROWTIME` may be specified as an explicit column but isn't required when you omit the column specifications. +!!! note + While streams will allow inserts where the KEY column is `null`, tables require a value for their + PRIMARY KEY and will reject any statement without one. + Example ------- The following statements are valid for a source with a schema like -`` with `KEY=KEY_COL`. +`ID INT KEY, COL_A VARCHAR`. ```sql --- inserts (1234, "key", "key", "A") -INSERT INTO foo (ROWTIME, ROWKEY, KEY_COL, COL_A) VALUES (1234, 'key', 'key', 'A'); +-- inserts (rowtime=1234, id="key", col_a="A") +INSERT INTO foo (ROWTIME, ID, COL_A) VALUES (1234, 'key', 'A'); + +-- also inserts (rowtime=1234, id="key", col_a="A") +INSERT INTO foo (COL_A, ID, ROWTIME) VALUES ('A', 'key', 1234); --- inserts (current_time(), "key", "key", "A") -INSERT INTO foo VALUES ('key', 'key', 'A'); +-- inserts (rowtime=current_time(), id="key", col_a="A") +INSERT INTO foo VALUES ('key', 'A'); --- inserts (current_time(), "key", "key", "A") +-- inserts (rowtime=current_time(), id="key", COL_A="A") INSERT INTO foo (KEY_COL, COL_A) VALUES ('key', 'A'); --- inserts (current_time(), "key", "key", null) +-- inserts (rowtime=current_time(), id="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. - -Page last revised on: {{ git_revision_date }} +Page last revised on: {{ git_revision_date }} diff --git a/docs-md/developer-guide/ksqldb-reference/select-pull-query.md b/docs-md/developer-guide/ksqldb-reference/select-pull-query.md index 3140eb8cc294..d16a38629979 100644 --- a/docs-md/developer-guide/ksqldb-reference/select-pull-query.md +++ b/docs-md/developer-guide/ksqldb-reference/select-pull-query.md @@ -15,7 +15,7 @@ Synopsis ```sql SELECT select_expr [, ...] FROM aggregate_table - WHERE ROWKEY=key + WHERE keyColumn=keyValue [AND window_bounds]; ``` @@ -26,7 +26,7 @@ Pulls the current value from the materialized table and terminates. The result of this statement isn't persisted in a Kafka topic and is printed out only in the console. -Pull queries enable you to fetch the current state of a materialized view. +Pull queries enable you to query the current state of a materialized view. Because materialized views are incrementally updated as new events arrive, pull queries run with predictably low latency. They're a great match for request/response flows. For asynchronous application flows, see @@ -35,7 +35,7 @@ request/response flows. For asynchronous application flows, see Execute a pull query by sending an HTTP request to the ksqlDB REST API, and the API responds with a single response. -The WHERE clause must contain a single value of `ROWKEY` to retrieve and may +The WHERE clause must contain a single value for the key of the row(s) to retrieve and may optionally include bounds on WINDOWSTART if the materialized table is windowed. Example diff --git a/docs-md/developer-guide/ksqldb-reference/select-push-query.md b/docs-md/developer-guide/ksqldb-reference/select-push-query.md index 1be83e403b29..58808d6225aa 100644 --- a/docs-md/developer-guide/ksqldb-reference/select-push-query.md +++ b/docs-md/developer-guide/ksqldb-reference/select-push-query.md @@ -47,8 +47,8 @@ In the previous statements, `from_item` is one of the following: - `table_name [ alias ]` - `from_item LEFT JOIN from_item ON join_condition` -The WHERE clause can refer to any column defined for a stream or table, -including the `ROWTIME` and `ROWKEY` system columns. +The WHERE clause can refer to any column defined for a stream or table, including the `ROWTIME` +system column. Example ------- diff --git a/docs-md/developer-guide/syntax-reference.md b/docs-md/developer-guide/syntax-reference.md index f0a2a6ff6813..28869e3e186a 100644 --- a/docs-md/developer-guide/syntax-reference.md +++ b/docs-md/developer-guide/syntax-reference.md @@ -406,7 +406,7 @@ The `KEY` property is optional. ksqlDB uses it as an optimization hint to determine if repartitioning can be avoided when performing aggregations and joins. -The type of the column named in the `KEY` property must match the type of the `ROWKEY` column. +The type of the column named in the `KEY` property must match the type of the key column. !!! important Don't set the KEY property, unless you have validated that your diff --git a/docs-md/developer-guide/test-and-debug/ksqldb-testing-tool.md b/docs-md/developer-guide/test-and-debug/ksqldb-testing-tool.md index b30c0fcd48ff..4b9fc84f7c70 100644 --- a/docs-md/developer-guide/test-and-debug/ksqldb-testing-tool.md +++ b/docs-md/developer-guide/test-and-debug/ksqldb-testing-tool.md @@ -61,8 +61,8 @@ are the supported statements in the testing tool: Here is a sample statements file for the testing tool: ```sql -CREATE STREAM orders (ROWKEY INT KEY, ORDERUNITS double) WITH (kafka_topic='test_topic', value_format='JSON'); -CREATE STREAM S1 AS SELECT ORDERUNITS, CASE WHEN orderunits < 2.0 THEN 'small' WHEN orderunits < 4.0 THEN 'medium' ELSE 'large' END AS case_result FROM orders EMIT CHANGES; +CREATE STREAM orders (ORDERID INT KEY, ORDERUNITS double) WITH (kafka_topic='test_topic', value_format='JSON'); +CREATE STREAM S1 AS SELECT ORDERUNITS, CASE WHEN orderunits < 2.0 THEN 'small' WHEN orderunits < 4.0 THEN 'medium' ELSE 'large' END AS size FROM orders EMIT CHANGES; ``` ### Input File @@ -77,11 +77,11 @@ the previous test: ```json { "inputs": [ - {"topic": "test_topic", "timestamp": 0, "value": {"ORDERUNITS": 2.0}, "key": 0}, - {"topic": "test_topic", "timestamp": 0, "value": {"ORDERUNITS": 4.0}, "key": 100}, - {"topic": "test_topic", "timestamp": 0, "value": {"ORDERUNITS": 6.0 }, "key": 101}, - {"topic": "test_topic", "timestamp": 0, "value": {"ORDERUNITS": 3.0}, "key": 101}, - {"topic": "test_topic", "timestamp": 0, "value": {"ORDERUNITS": 1.0}, "key": 101} + {"topic": "test_topic", "timestamp": 0, "key": 99, "value": {"ORDERUNITS": 2.0}}, + {"topic": "test_topic", "timestamp": 10, "key": 100, "value": {"ORDERUNITS": 4.0}}, + {"topic": "test_topic", "timestamp": 11, "key": 101, "value": {"ORDERUNITS": 6.0 }}, + {"topic": "test_topic", "timestamp": 12, "key": 102, "value": {"ORDERUNITS": 3.0}}, + {"topic": "test_topic", "timestamp": 9, "key": 106, "value": {"ORDERUNITS": 1.0}} ] } ``` @@ -99,11 +99,11 @@ expected output file for the previous test: ```json { "outputs": [ - {"topic": "S1", "timestamp": 0, "value": {"ORDERUNITS": 2.0, "CASE_RESULT": "medium"}, "key": 0}, - {"topic": "S1", "timestamp": 0, "value": {"ORDERUNITS": 4.0, "CASE_RESULT": "large"}, "key": 100}, - {"topic": "S1", "timestamp": 0, "value": {"ORDERUNITS": 6.0, "CASE_RESULT": "large"}, "key": 101}, - {"topic": "S1", "timestamp": 0, "value": {"ORDERUNITS": 3.0, "CASE_RESULT": "medium"}, "key": 101}, - {"topic": "S1", "timestamp": 0, "value": {"ORDERUNITS": 1.0, "CASE_RESULT": "small"},"key": 101} + {"topic": "S1", "timestamp": 0, "key": 99, "value": {"ORDERUNITS": 2.0, "SIZE": "medium"}}, + {"topic": "S1", "timestamp": 10, "key": 100, "value": {"ORDERUNITS": 4.0, "SIZE": "large"}}, + {"topic": "S1", "timestamp": 11, "key": 101, "value": {"ORDERUNITS": 6.0, "SIZE": "large"}}, + {"topic": "S1", "timestamp": 12, "key": 102, "value": {"ORDERUNITS": 3.0, "SIZE": "medium"}}, + {"topic": "S1", "timestamp": 9,"key": 106, "value": {"ORDERUNITS": 1.0, "SIZE": "small"}} ] } ``` @@ -114,7 +114,7 @@ specify a window for a message you can add a `window` field to the message. A window field has three fields: - **start:** the start time for the window. -- **end:** the end time for the window. +- **end:** the end time for the window. (ignored if not a session window) - **type:** the type of the window. A window type can be `time` or `session`. diff --git a/docs-md/operate-and-deploy/installation/server-config/avro-schema.md b/docs-md/operate-and-deploy/installation/server-config/avro-schema.md index b80c7a871698..8c21e15a73d7 100644 --- a/docs-md/operate-and-deploy/installation/server-config/avro-schema.md +++ b/docs-md/operate-and-deploy/installation/server-config/avro-schema.md @@ -113,16 +113,15 @@ Note: the Avro schema must be registered in the {{ site.sr }} under the subject `users-avro-topic-value`. By default, the key of the data will be assumed to be a single `KAFKA` -serialized `STRING` called `ROWKEY`. If the key schema differs, then you -can provide just the key column in the statement. For example, the following -creates the `users` table with a 64-bit integer key and infers the value -columns from the Avro schema. +serialized `STRING` called `ROWKEY`. ksqlDB does not support currently +support key formats other than `KAFKA`. However, you can explicitly specify the type and name of the +key column in the statement. For example, the following creates the `users` table with a 64-bit +integer key column, called `userId`, and infers the value columns from the Avro schema. ```sql -CREATE TABLE users (ROWKEY BIGINT KEY) +CREATE TABLE users (userId BIGINT KEY) WITH (KAFKA_TOPIC='users-avro-topic', - VALUE_FORMAT='AVRO', - KEY='userid'); + VALUE_FORMAT='AVRO'); ``` @@ -138,7 +137,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 (userId BIGINT KEY, viewtime BIGINT, pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews-avro-topic', VALUE_FORMAT='AVRO'); ``` @@ -157,7 +156,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 (userId BIGINT KEY, viewtime BIGINT, pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews_kafka_topic_json', VALUE_FORMAT='JSON'); CREATE STREAM pageviews_avro diff --git a/docs-md/tutorials/embedded-connect.md b/docs-md/tutorials/embedded-connect.md index 95a49b651e45..f94878ed9242 100644 --- a/docs-md/tutorials/embedded-connect.md +++ b/docs-md/tutorials/embedded-connect.md @@ -250,15 +250,14 @@ columns. ```sql CREATE TABLE driverProfiles ( - rowkey INTEGER KEY, - driver_id INTEGER, + driver_id INTEGER KEY make STRING, model STRING, year INTEGER, license_plate STRING, rating DOUBLE ) -WITH (kafka_topic='jdbc_driver_profiles', value_format='json', key='driver_id'); +WITH (kafka_topic='jdbc_driver_profiles', value_format='json'); ``` Tables in ksqlDB support update semantics, where each message in the @@ -284,21 +283,19 @@ continuous stream of location updates. ```sql CREATE STREAM driverLocations ( - rowkey INTEGER KEY, - driver_id INTEGER, + driver_id INTEGER KEY, latitude DOUBLE, longitude DOUBLE, speed DOUBLE ) -WITH (kafka_topic='driver_locations', value_format='json', partitions=1, key='driver_id'); +WITH (kafka_topic='driver_locations', value_format='json', partitions=1); CREATE STREAM riderLocations ( - rowkey INTEGER KEY, - driver_id INTEGER, + driver_id INTEGER KEY, latitude DOUBLE, longitude DOUBLE ) -WITH (kafka_topic='rider_locations', value_format='json', partitions=1, key='driver_id'); +WITH (kafka_topic='rider_locations', value_format='json', partitions=1); ``` 11. Enrich driverLocations stream by joining with PostgreSQL data @@ -317,7 +314,6 @@ the `driver_profiles` table stored in PostgreSQL. ```sql CREATE STREAM enrichedDriverLocations AS SELECT - dl.driver_id AS driver_id, dl.latitude AS latitude, dl.longitude AS longitude, dl.speed AS speed, diff --git a/docs-md/tutorials/examples.md b/docs-md/tutorials/examples.md index 12f9ef4db035..f9e765f495d9 100644 --- a/docs-md/tutorials/examples.md +++ b/docs-md/tutorials/examples.md @@ -33,8 +33,7 @@ serializers. ksqlDB supports `INT`, `BIGINT`, `DOUBLE`, and `STRING` key types. ```sql CREATE STREAM pageviews - (rowkey STRING KEY, - viewtime BIGINT, + (viewtime BIGINT, userid VARCHAR, pageid VARCHAR) WITH (KAFKA_TOPIC='pageviews', @@ -45,19 +44,18 @@ CREATE STREAM pageviews The previous statement doesn't make any assumptions about the Kafka message key in the underlying {{ site.ak }} topic. But if the value of the message key -in {{ site.aktm }} is the same as one of the columns defined in the stream in -ksqlDB, you can provide this information in the WITH clause. For example, if -the {{ site.aktm }} message key has the same value as the `pageid` column, you +in {{ site.aktm }} contains a `KAFKA` serialized primitive, you can can include +this in the schema of the stream. For example, if the {{ site.aktm }} message key +contained the `pageid`, rather than it being in a field in the value, you can write the CREATE STREAM statement like this: ```sql CREATE STREAM pageviews - (viewtime BIGINT, - userid VARCHAR, - pageid VARCHAR) + (pageid VARCHAR KEY, + viewtime BIGINT, + userid VARCHAR) WITH (KAFKA_TOPIC='pageviews', - VALUE_FORMAT='DELIMITED', - KEY='pageid'); + VALUE_FORMAT='DELIMITED'); ``` ### Associate {{ site.aktm }} message timestamps @@ -71,12 +69,11 @@ the message timestamp, you can rewrite the above statement like this: ```sql CREATE STREAM pageviews - (viewtime BIGINT, - userid VARCHAR, - pageid VARCHAR) + (pageid VARCHAR KEY, + viewtime BIGINT, + userid VARCHAR) WITH (KAFKA_TOPIC='pageviews', VALUE_FORMAT='DELIMITED', - KEY='pageid', TIMESTAMP='viewtime'); ``` @@ -94,10 +91,10 @@ column of `map` type: ```sql CREATE TABLE users - (registertime BIGINT, + (userid VARCHAR KEY, + registertime BIGINT, gender VARCHAR, regionid VARCHAR, - userid VARCHAR, interests array, contactinfo map) WITH (KAFKA_TOPIC='users', @@ -105,9 +102,6 @@ CREATE TABLE users KEY = 'userid'); ``` -Note that specifying KEY is required in table declaration, see -[Key Requirements](../developer-guide/syntax-reference.md#key-requirements). - Working with streams and tables ------------------------------- diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index dba350ed959f..9d79c72dd2d4 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -85,6 +85,7 @@ import io.confluent.ksql.util.HandlerMaps.ClassHandlerMap1; import io.confluent.ksql.util.HandlerMaps.Handler1; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.SchemaUtil; import io.confluent.ksql.util.TabularRow; import java.io.Closeable; import java.io.File; @@ -454,11 +455,12 @@ private static String formatFieldType( final Optional windowType, final String keyField ) { - if (field.getName().equals("ROWTIME")) { + if (field.getName().equals(SchemaUtil.ROWTIME_NAME.text())) { return String.format("%-16s %s", field.getSchema().toTypeString(), "(system)"); } - if (field.getName().equals("ROWKEY")) { + // Todo(ac): + if (field.getName().equals(SchemaUtil.ROWKEY_NAME.text())) { final String wt = windowType .map(v -> " (Window type: " + v + ")") .orElse(""); diff --git a/ksql-common/src/main/java/io/confluent/ksql/schema/ksql/LogicalSchema.java b/ksql-common/src/main/java/io/confluent/ksql/schema/ksql/LogicalSchema.java index 1d4a645d6ad2..993c8f304cd5 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/schema/ksql/LogicalSchema.java +++ b/ksql-common/src/main/java/io/confluent/ksql/schema/ksql/LogicalSchema.java @@ -306,20 +306,28 @@ private static ConnectSchema toConnectSchema( public static class Builder { - private final ImmutableList.Builder explicitColumns = ImmutableList.builder(); - + private final ImmutableList.Builder columns = ImmutableList.builder(); private final Set seenKeys = new HashSet<>(); private final Set seenValues = new HashSet<>(); + // Todo(ac): remove private boolean addImplicitRowKey = true; private boolean addImplicitRowTime = true; + // Todo(ac): remove public Builder noImplicitColumns() { addImplicitRowKey = false; addImplicitRowTime = false; return this; } + // Todo(ac): Test. + public Builder withRowTime() { + addImplicitRowTime = false; + columns.add(IMPLICIT_TIME_COLUMN); + return this; + } + public Builder keyColumns(final Iterable columns) { columns.forEach(this::keyColumn); return this; @@ -359,7 +367,7 @@ public LogicalSchema build() { allColumns.add(IMPLICIT_KEY_COLUMN); } - allColumns.addAll(explicitColumns.build()); + allColumns.addAll(columns.build()); return new LogicalSchema(allColumns.build()); } @@ -383,7 +391,7 @@ private void addColumn(final Column column) { break; } - explicitColumns.add(column); + columns.add(column); } } } \ No newline at end of file diff --git a/ksql-common/src/test/java/io/confluent/ksql/schema/ksql/LogicalSchemaTest.java b/ksql-common/src/test/java/io/confluent/ksql/schema/ksql/LogicalSchemaTest.java index 0c5bdde424f1..aeb5b9e8e28d 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/schema/ksql/LogicalSchemaTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/schema/ksql/LogicalSchemaTest.java @@ -461,8 +461,9 @@ public void shouldAddMetaAndKeyColumnsOnlyOnce() { public void shouldRemoveOthersWhenAddingMetasAndKeyColumns() { // Given: final LogicalSchema ksqlSchema = LogicalSchema.builder() + .keyColumn(K0, STRING) .valueColumn(F0, BIGINT) - .valueColumn(ROWKEY_NAME, DOUBLE) + .valueColumn(K0, DOUBLE) .valueColumn(F1, BIGINT) .valueColumn(ROWTIME_NAME, DOUBLE) .build(); @@ -472,10 +473,11 @@ public void shouldRemoveOthersWhenAddingMetasAndKeyColumns() { // Then: assertThat(result, is(LogicalSchema.builder() + .keyColumn(K0, STRING) .valueColumn(F0, BIGINT) .valueColumn(F1, BIGINT) .valueColumn(ROWTIME_NAME, BIGINT) - .valueColumn(ROWKEY_NAME, STRING) + .valueColumn(K0, STRING) .build() )); } @@ -524,8 +526,9 @@ public void shouldRemoveWindowedMetaColumnsFromValue() { public void shouldRemoveMetaColumnsWhereEverTheyAre() { // Given: final LogicalSchema schema = LogicalSchema.builder() + .keyColumn(K0, STRING) .valueColumn(F0, BIGINT) - .valueColumn(ROWKEY_NAME, STRING) + .valueColumn(K0, STRING) .valueColumn(F1, BIGINT) .valueColumn(ROWTIME_NAME, BIGINT) .build(); @@ -535,6 +538,7 @@ public void shouldRemoveMetaColumnsWhereEverTheyAre() { // Then: assertThat(result, is(LogicalSchema.builder() + .keyColumn(K0, STRING) .valueColumn(F0, BIGINT) .valueColumn(F1, BIGINT) .build() diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analysis.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analysis.java index d8aa5c2ea8d9..aee2d13563c5 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analysis.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analysis.java @@ -37,6 +37,7 @@ import io.confluent.ksql.parser.tree.WithinExpression; import io.confluent.ksql.planner.plan.JoinNode; import io.confluent.ksql.planner.plan.JoinNode.JoinType; +import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.util.SchemaUtil; @@ -58,6 +59,7 @@ public class Analysis implements ImmutableAnalysis { private final ResultMaterialization resultMaterialization; private final Function, SourceSchemas> sourceSchemasFactory; private Optional into = Optional.empty(); + private final Set fromKeyColumns = new HashSet<>(); private final List fromDataSources = new ArrayList<>(); private Optional joinInfo = Optional.empty(); private Optional whereExpression = Optional.empty(); @@ -190,6 +192,10 @@ public List getFromDataSources() { return ImmutableList.copyOf(fromDataSources); } + Set getKeyColumnNames() { + return ImmutableSet.copyOf(fromKeyColumns); + } + @Override public SourceSchemas getFromSourceSchemas(final boolean postAggregate) { final Map schemaBySource = fromDataSources.stream() @@ -206,6 +212,10 @@ void addDataSource(final SourceName alias, final DataSource dataSource) { throw new IllegalArgumentException("Data source type not supported yet: " + dataSource); } + dataSource.getSchema().key().stream() + .map(Column::name) + .forEach(fromKeyColumns::add); + fromDataSources.add(new AliasedDataSource(alias, dataSource)); } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index 989c1b80f46d..a9ac59b718f6 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -576,7 +576,7 @@ private void visitSelectStar(final AllColumns allColumns) { // See https://github.com/confluentinc/ksql/issues/3731 for more info final List valueColumns = persistent && !analysis.isJoin() ? schema.value() - : systemColumnsToTheFront(schema.withMetaAndKeyColsInValue(windowed).value()); + : orderColumns(schema.withMetaAndKeyColsInValue(windowed).value(), schema); for (final Column column : valueColumns) { @@ -596,12 +596,15 @@ private void visitSelectStar(final AllColumns allColumns) { } } - private List systemColumnsToTheFront(final List columns) { - // When doing a `select *` the system columns should be at the front of the column list + private List orderColumns( + final List columns, + final LogicalSchema schema + ) { + // When doing a `select *` system and key columns should be at the front of the column list // but are added at the back during processing for performance reasons. // Switch them around here: - final Map> partitioned = columns.stream() - .collect(Collectors.groupingBy(c -> SchemaUtil.isSystemColumn(c.name()))); + final Map> partitioned = columns.stream().collect(Collectors + .groupingBy(c -> SchemaUtil.isSystemColumn(c.name()) || schema.isKeyColumn(c.name()))); final List all = partitioned.get(true); all.addAll(partitioned.get(false)); @@ -639,6 +642,13 @@ private void addSelectItem(final Expression exp, final ColumnName columnName) { throw new KsqlException("Reserved column name in select: " + columnName + ". " + "Please remove or alias the column."); } + + // Todo(ac): This needs to check against the key columns of the RESTUL schema. + if (analysis.getKeyColumnNames().contains(columnName)) { + throw new KsqlException("Value column name " + + "'" + columnName.text() + "' " + + "clashes with key column name. Please remove or alias the column."); + } } final Set columnNames = new HashSet<>(); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/PullQueryValidator.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/PullQueryValidator.java index 8ec4747a7ad2..d33c443a4db8 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/PullQueryValidator.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/PullQueryValidator.java @@ -46,9 +46,9 @@ public class PullQueryValidator implements QueryValidator { + System.lineSeparator() + "For example, the following are pull queries:" + System.lineSeparator() - + "\t'SELECT * FROM X WHERE ROWKEY=Y;' (non-windowed table)" + + "\t'SELECT * FROM X WHERE >=Y;' (non-windowed table)" + System.lineSeparator() - + "\t'SELECT * FROM X WHERE ROWKEY=Y AND WINDOWSTART>=Z;' (windowed table)" + + "\t'SELECT * FROM X WHERE =Y AND WINDOWSTART>=Z;' (windowed table)" + System.lineSeparator() + System.lineSeparator() + "The following is a push query:" diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java index 8ce37cdd3421..b9fe85df87b4 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java @@ -177,12 +177,7 @@ private static LogicalSchema buildSchema(final TableElements tableElements) { throw new KsqlException("'" + e.getName().text() + "' is a reserved column name."); } - if (e.getNamespace() == Namespace.KEY) { - if (!isRowKey) { - throw new KsqlException("'" + e.getName().text() + "' is an invalid KEY column name. " - + "KSQL currently only supports KEY columns named ROWKEY."); - } - } else if (isRowKey) { + if (isRowKey && e.getNamespace() != Namespace.KEY) { throw new KsqlException("'" + e.getName().text() + "' is a reserved column name. " + "It can only be used for KEY columns."); } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index e1f2170d305c..06d03f7cab5e 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -37,7 +37,6 @@ import io.confluent.ksql.rest.SessionProperties; import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; -import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.ksql.SchemaConverters; @@ -62,7 +61,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.function.LongSupplier; @@ -143,6 +141,7 @@ private InsertValuesExecutor( this.valueSerdeFactory = Objects.requireNonNull(valueSerdeFactory, "valueSerdeFactory"); } + @SuppressWarnings("unused") // Part of required API. public void execute( final ConfiguredStatement statement, final SessionProperties sessionProperties, @@ -176,7 +175,7 @@ public void execute( } } - private DataSource getDataSource( + private static DataSource getDataSource( final KsqlConfig ksqlConfig, final MetaStore metaStore, final InsertValues insertValues @@ -267,11 +266,23 @@ private RowData extractRow( final Map values = resolveValues( insertValues, columns, schema, functionRegistry, config); - handleExplicitKeyField(values, dataSource.getKeyField()); + handleExplicitKeyField( + values, + dataSource.getKeyField(), + Iterables.getOnlyElement(dataSource.getSchema().key()) + ); + + if (dataSource.getDataSourceType() == DataSourceType.KTABLE) { + final String noValue = dataSource.getSchema().key().stream() + .map(Column::name) + .filter(colName -> !values.containsKey(colName)) + .map(ColumnName::text) + .collect(Collectors.joining(", ")); - if (dataSource.getDataSourceType() == DataSourceType.KTABLE - && values.get(SchemaUtil.ROWKEY_NAME) == null) { - throw new KsqlException("Value for ROWKEY is required for tables"); + if (!noValue.isEmpty()) { + throw new KsqlException("Value for primary key column(s) " + + noValue + " is required for tables"); + } } final long ts = (long) values.getOrDefault(SchemaUtil.ROWTIME_NAME, clock.getAsLong()); @@ -358,26 +369,29 @@ private static Map resolveValues( private static void handleExplicitKeyField( final Map values, - final KeyField keyField + final KeyField keyField, + final Column keyColumn ) { - final Optional keyFieldName = keyField.ref(); - if (keyFieldName.isPresent()) { - final ColumnName key = keyFieldName.get(); - final Object keyValue = values.get(key); - final Object rowKeyValue = values.get(SchemaUtil.ROWKEY_NAME); - - if (keyValue != null ^ rowKeyValue != null) { - if (keyValue == null) { - values.put(key, rowKeyValue); + // 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(SchemaUtil.ROWKEY_NAME, keyValue); + values.put(keyColumnName, keyFieldValue); } - } else if (keyValue != null && !Objects.equals(keyValue, rowKeyValue)) { - throw new KsqlException(String.format( - "Expected ROWKEY and %s to match but got %s and %s respectively.", - key.toString(FormatOptions.noEscape()), rowKeyValue, keyValue)); + } 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) { diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java index a29f0e06404e..82d3723764df 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java @@ -423,6 +423,9 @@ private LogicalSchema buildAggregateSchema( sourcePlanNode.getSelectExpressions() ); + // Todo(ac): Support getting key name from GROUP BY + // Todo(ac): Support aliasing in GROUP BY to provide key column name? + // Todo(ac): Update partition-data.md once done. return LogicalSchema.builder() .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) .valueColumns(sourceSchema.value()) @@ -440,6 +443,9 @@ private LogicalSchema buildRepartitionedSchema( final SqlType keyType = typeManager.getExpressionSqlType(partitionBy); + // Todo(ac): Support getting key name from partition by column + // Todo(ac): Support aliasing in PARTITION BY to provide key column name? + // Todo(ac): Update partition-data.md once done. return LogicalSchema.builder() .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) .valueColumns(sourceSchema.value()) diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java index 4b001b830d2b..b7f65709c590 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java @@ -297,6 +297,7 @@ List resolveGroupByExpressions( final boolean specialRowTimeHandling = !(aggregateArgExpanded instanceof SchemaKTable); final Function mapper = e -> { + // Todo(ac): final boolean rowKey = e instanceof UnqualifiedColumnReferenceExp && ((UnqualifiedColumnReferenceExp) e).getReference().equals( SchemaUtil.ROWKEY_NAME); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java b/ksql-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java index 7e9ffb196e04..fcf667c64e71 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java @@ -449,6 +449,11 @@ private static LogicalSchema buildTransientQuerySchema(final LogicalSchema fullS final LogicalSchema.Builder builder = LogicalSchema.builder() .noImplicitColumns(); + // Todo(ac): Don't just return value schema. Return true schema with KEY columns + // If we pass namespace info to CLI this will also allow CLI to id any KEY column(s) + // Returned schema should only include ROWTIME if pass schema has it in the value. + // Returned schema should only include key columns(s)? if pass schema has it in the value. + builder.valueColumns(fullSchema.value()); return builder.build(); } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java index 7d81e1c7f1dc..8babda914dc7 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java @@ -358,7 +358,7 @@ private KeyField getNewKeyField(final Expression expression) { } final ColumnName columnName = ((UnqualifiedColumnReferenceExp) expression).getReference(); - final KeyField newKeyField = isRowKey(columnName) ? keyField : KeyField.of(columnName); + final KeyField newKeyField = isKeyColumn(columnName) ? keyField : KeyField.of(columnName); return getSchema().isMetaColumn(columnName) ? KeyField.none() : newKeyField; } @@ -381,12 +381,12 @@ protected boolean repartitionNotNeeded(final Expression expression) { .map(kf -> kf.name().equals(proposedKey.name())) .orElse(false); - return namesMatch || isRowKey(columnName); + return namesMatch || isKeyColumn(columnName); } - private boolean isRowKey(final ColumnName fieldName) { - // until we support structured keys, there will never be any key column other - // than "ROWKEY" - furthermore, that key column is always prefixed at this point + private boolean isKeyColumn(final ColumnName fieldName) { + // until we support structured keys, there will only be a single key column + // - furthermore, that key column is always prefixed at this point // unless it is a join, in which case every other source field is prefixed return fieldName.equals(schema.key().get(0).name()); } @@ -409,6 +409,7 @@ private boolean rekeyRequired(final List groupByExpressions) { return true; } + // Todo(ac): if (groupByField.equals(SchemaUtil.ROWKEY_NAME)) { return false; } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java index 3f9b3c5fa51d..1fdb0f565010 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java @@ -15,6 +15,7 @@ package io.confluent.ksql.structured; +import com.google.common.collect.Iterables; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.context.QueryContext.Stacker; @@ -147,8 +148,8 @@ public SchemaKStream selectKey(final Expression keyExpression, } throw new UnsupportedOperationException("Cannot repartition a TABLE source. " - + "If this is a join, make sure that the criteria uses the TABLE key " - + this.keyField.ref().map(ColumnName::toString).orElse("ROWKEY") + " instead of " + + "If this is a join, make sure that the criteria uses the TABLE's key column " + + Iterables.getOnlyElement(schema.key()).name().text() + " instead of " + keyExpression); } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java index 3f31b2adce3a..e58029927db6 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java @@ -35,16 +35,24 @@ public class SourceSchemasTest { private static final SourceName ALIAS_1 = SourceName.of("S1"); private static final SourceName ALIAS_2 = SourceName.of("S2"); - private static final ColumnName COMMON_FIELD_NAME = ColumnName.of("F0"); + + private static final ColumnName K0 = ColumnName.of("K0"); + private static final ColumnName K1 = ColumnName.of("K1"); + + private static final ColumnName COMMON_VALUE_FIELD_NAME = ColumnName.of("V0"); + private static final ColumnName V1 = ColumnName.of("V1"); + private static final ColumnName V2 = ColumnName.of("V2"); private static final LogicalSchema SCHEMA_1 = LogicalSchema.builder() - .valueColumn(COMMON_FIELD_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("F1"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.INTEGER) + .valueColumn(COMMON_VALUE_FIELD_NAME, SqlTypes.STRING) + .valueColumn(V1, SqlTypes.STRING) .build(); private static final LogicalSchema SCHEMA_2 = LogicalSchema.builder() - .valueColumn(COMMON_FIELD_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("F2"), SqlTypes.STRING) + .keyColumn(K1, SqlTypes.STRING) + .valueColumn(COMMON_VALUE_FIELD_NAME, SqlTypes.STRING) + .valueColumn(V2, SqlTypes.STRING) .build(); private SourceSchemas sourceSchemas; @@ -84,28 +92,28 @@ public void shouldFindNoField() { @Test public void shouldFindNoQualifiedField() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), ColumnName.of("F2")), is(empty())); + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), V2), is(empty())); } @Test public void shouldFindUnqualifiedUniqueField() { - assertThat(sourceSchemas.sourcesWithField(Optional.empty(), ColumnName.of("F1")), contains(ALIAS_1)); + assertThat(sourceSchemas.sourcesWithField(Optional.empty(), V1), contains(ALIAS_1)); } @Test public void shouldFindQualifiedUniqueField() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_2), ColumnName.of("F2")), contains(ALIAS_2)); + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_2), V2), contains(ALIAS_2)); } @Test public void shouldFindUnqualifiedCommonField() { - assertThat(sourceSchemas.sourcesWithField(Optional.empty(), COMMON_FIELD_NAME), + assertThat(sourceSchemas.sourcesWithField(Optional.empty(), COMMON_VALUE_FIELD_NAME), containsInAnyOrder(ALIAS_1, ALIAS_2)); } @Test public void shouldFindQualifiedFieldOnlyInThatSource() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), COMMON_FIELD_NAME), + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), COMMON_VALUE_FIELD_NAME), contains(ALIAS_1)); } @@ -121,22 +129,28 @@ public void shouldMatchNonValueFieldNameIfAliaasedMetaField() { @Test public void shouldMatchNonValueFieldNameIfKeyField() { - assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), SchemaUtil.ROWKEY_NAME), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), K0), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), K1), is(true)); + } + + @Test + public void shouldNotMatchNonKeyFieldOnWrongSource() { + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), K0), is(false)); } @Test public void shouldMatchNonValueFieldNameIfAliasedKeyField() { - assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), SchemaUtil.ROWKEY_NAME), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), K1), is(true)); } @Test(expected = IllegalArgumentException.class) public void shouldThrowOnUnknownSourceWhenMatchingNonValueFields() { - sourceSchemas.matchesNonValueField(Optional.of(SourceName.of("unknown")), SchemaUtil.ROWKEY_NAME); + sourceSchemas.matchesNonValueField(Optional.of(SourceName.of("unknown")), K0); } @Test public void shouldNotMatchOtherFields() { - assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), ColumnName.of("F2")), is(false)); + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), V2), is(false)); } @Test diff --git a/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java b/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java index dc9782d09517..18be0b459fce 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java @@ -66,7 +66,6 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collections; @@ -88,7 +87,7 @@ public class CodeGenRunnerTest { private static final String COL_INVALID_JAVA = "col!Invalid:("; private static final LogicalSchema META_STORE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("COL1"), SqlTypes.STRING) .valueColumn(ColumnName.of("COL2"), SqlTypes.STRING) diff --git a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java index b0be612a085d..cea5b092b8d3 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java @@ -858,7 +858,7 @@ public void shouldAllowNonStringKeyColumn() { } @Test - public void shouldThrowOnKeyColumnThatIsNotCalledRowKey() { + public void shouldNotThrowOnKeyColumnThatIsNotCalledRowKey() { // Given: final CreateStream statement = new CreateStream( SOME_NAME, @@ -867,13 +867,14 @@ public void shouldThrowOnKeyColumnThatIsNotCalledRowKey() { withProperties ); - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("'someKey' is an invalid KEY column name. " - + "KSQL currently only supports KEY columns named ROWKEY."); - // When: - createSourceFactory.createStreamCommand(statement, ksqlConfig); + final CreateStreamCommand result = createSourceFactory + .createStreamCommand(statement, ksqlConfig); + + // Then: + assertThat(result.getSchema().key(), contains( + keyColumn(ColumnName.of("someKey"), SqlTypes.STRING) + )); } private void givenProperty(final String name, final Literal value) { diff --git a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java index 699edb9a4efd..a3e41061ffd7 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.serde.WindowInfo; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import java.util.Set; import org.hamcrest.MatcherAssert; @@ -46,7 +45,7 @@ public class DdlCommandExecTest { private static final SourceName TABLE_NAME = SourceName.of("t1"); private static final String TOPIC_NAME = "topic"; private static final LogicalSchema SCHEMA = new LogicalSchema.Builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("F1"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("F2"), SqlTypes.STRING) .build(); diff --git a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java index 61ab3c78f284..c9a1a70b772d 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.GenericRow; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; @@ -67,6 +68,7 @@ import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.SchemaUtil; import java.math.BigDecimal; import java.math.MathContext; import java.util.Collections; @@ -97,19 +99,26 @@ @RunWith(MockitoJUnitRunner.class) public class InsertValuesExecutorTest { + private static final ColumnName K0 = ColumnName.of("k0"); private static final ColumnName COL0 = ColumnName.of("COL0"); - private static final LogicalSchema SINGLE_FIELD_SCHEMA = LogicalSchema.builder() + private static final ColumnName COL1 = ColumnName.of("COL1"); + private static final ColumnName INT_COL = ColumnName.of("INT"); + + private static final LogicalSchema SINGLE_VALUE_COLUMN_SCHEMA = LogicalSchema.builder() + .keyColumn(K0, SqlTypes.STRING) .valueColumn(COL0, SqlTypes.STRING) .build(); private static final LogicalSchema SCHEMA = LogicalSchema.builder() + .keyColumn(K0, SqlTypes.STRING) .valueColumn(COL0, SqlTypes.STRING) - .valueColumn(ColumnName.of("COL1"), SqlTypes.BIGINT) + .valueColumn(COL1, SqlTypes.BIGINT) .build(); private static final LogicalSchema BIG_SCHEMA = LogicalSchema.builder() - .valueColumn(COL0, SqlTypes.STRING) // named COL0 for auto-ROWKEY - .valueColumn(ColumnName.of("INT"), SqlTypes.INTEGER) + .keyColumn(K0, SqlTypes.STRING) + .valueColumn(COL0, SqlTypes.STRING) + .valueColumn(INT_COL, SqlTypes.INTEGER) .valueColumn(ColumnName.of("BIGINT"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("DOUBLE"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("BOOLEAN"), SqlTypes.BOOLEAN) @@ -203,10 +212,10 @@ public void shouldHandleFullRow() { @Test public void shouldInsertWrappedSingleField() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.of(COL0)); final ConfiguredStatement statement = givenInsertValues( - valueFieldNames(SINGLE_FIELD_SCHEMA), + valueFieldNames(SINGLE_VALUE_COLUMN_SCHEMA), ImmutableList.of(new StringLiteral("new")) ); @@ -223,13 +232,13 @@ public void shouldInsertWrappedSingleField() { public void shouldInsertUnwrappedSingleField() { // Given: givenSourceStreamWithSchema( - SINGLE_FIELD_SCHEMA, + SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.of(SerdeOption.UNWRAP_SINGLE_VALUES), Optional.of(COL0)) ; final ConfiguredStatement statement = givenInsertValues( - valueFieldNames(SINGLE_FIELD_SCHEMA), + valueFieldNames(SINGLE_VALUE_COLUMN_SCHEMA), ImmutableList.of(new StringLiteral("new")) ); @@ -245,8 +254,8 @@ public void shouldInsertUnwrappedSingleField() { @Test public void shouldFillInRowtime() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("str"), new StringLiteral("str"), @@ -264,10 +273,10 @@ public void shouldFillInRowtime() { } @Test - public void shouldHandleRowTimeWithoutRowKey() { + public void shouldHandleRowTimeWithoutKey() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWTIME", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(SchemaUtil.ROWTIME_NAME, COL0, COL1), ImmutableList.of( new LongLiteral(1234L), new StringLiteral("str"), @@ -285,10 +294,10 @@ public void shouldHandleRowTimeWithoutRowKey() { } @Test - public void shouldFillInRowKeyFromSpecifiedKey() { + public void shouldFillInKeyColumnFromSpecifiedKeyField() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L) @@ -328,8 +337,8 @@ public void shouldFillInFullRowWithNoSchema() { @Test public void shouldFillInMissingColumnsWithNulls() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0), ImmutableList.of( new StringLiteral("str"), new StringLiteral("str")) @@ -347,8 +356,8 @@ public void shouldFillInMissingColumnsWithNulls() { @Test public void shouldFillInKeyFromRowKey() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L) @@ -367,8 +376,8 @@ public void shouldFillInKeyFromRowKey() { @Test public void shouldHandleOutOfOrderSchema() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1, COL0), ImmutableList.of( new LongLiteral(2L), new StringLiteral("str") @@ -387,8 +396,8 @@ public void shouldHandleOutOfOrderSchema() { @Test public void shouldHandleAllSortsOfLiterals() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1, COL0), ImmutableList.of( new LongLiteral(2L), new StringLiteral("str")) @@ -438,10 +447,10 @@ public void shouldHandleNullKeyForSourceWithKeyField() { @Test public void shouldHandleNegativeValueExpression() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(ColumnName.of("COL0"))); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), ArithmeticUnaryExpression.negative(Optional.empty(), new LongLiteral(1)) @@ -460,10 +469,10 @@ public void shouldHandleNegativeValueExpression() { @Test public void shouldHandleUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0), ImmutableList.of( new FunctionCall( FunctionName.of("SUBSTRING"), @@ -481,10 +490,10 @@ public void shouldHandleUdfs() { @Test public void shouldHandleNestedUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0), ImmutableList.of( new FunctionCall( FunctionName.of("SUBSTRING"), @@ -510,8 +519,8 @@ public void shouldAllowUpcast() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new IntegerLiteral(1) @@ -682,8 +691,8 @@ public void shouldThrowOnTopicAuthorizationException() { @Test public void shouldThrowIfRowKeyAndKeyDoNotMatch() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0), ImmutableList.of( new StringLiteral("foo"), new StringLiteral("bar")) @@ -691,7 +700,7 @@ public void shouldThrowIfRowKeyAndKeyDoNotMatch() { // Expect: expectedException.expect(KsqlException.class); - expectedException.expectCause(hasMessage(containsString("Expected ROWKEY and COL0 to match"))); + expectedException.expectCause(hasMessage(containsString("Expected k0 and COL0 to match"))); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -719,8 +728,8 @@ public void shouldFailOnDowncast() { // Given: givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("INT"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(INT_COL), ImmutableList.of( new DoubleLiteral(1.1) ) @@ -739,8 +748,8 @@ public void shouldHandleStreamsWithNoKeyField() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("key"), new StringLiteral("str"), @@ -761,8 +770,8 @@ public void shouldHandleTablesWithNoKeyField() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("key"), new StringLiteral("str"), @@ -783,8 +792,8 @@ public void shouldHandleStreamsWithNoKeyFieldAndNoRowKeyProvided() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L)) @@ -804,8 +813,8 @@ public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L)) @@ -814,7 +823,7 @@ public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage( - "Failed to insert values into 'TOPIC'. Value for ROWKEY is required for tables"); + "Failed to insert values into 'TOPIC'. Value for primary key column(s) k0 is required for tables"); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -825,8 +834,8 @@ public void shouldThrowOnTablesWithKeyFieldAndNullKeyFieldValueProvided() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1), ImmutableList.of( new LongLiteral(2L)) ); @@ -834,7 +843,7 @@ public void shouldThrowOnTablesWithKeyFieldAndNullKeyFieldValueProvided() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage( - "Failed to insert values into 'TOPIC'. Value for ROWKEY is required for tables"); + "Failed to insert values into 'TOPIC'. Value for primary key column(s) k0 is required for tables"); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -874,13 +883,6 @@ public void shouldBuildCorrectSerde() { ); } - private static ConfiguredStatement givenInsertValuesStrings( - final List columns, - final List values - ) { - return givenInsertValues(columns.stream().map(ColumnName::of).collect(Collectors.toList()), values); - } - private static ConfiguredStatement givenInsertValues( final List columns, final List values @@ -923,7 +925,7 @@ private void givenDataSourceWithSchema( ); final KeyField valueKeyField = keyField - .map(kf -> KeyField.of(kf)) + .map(KeyField::of) .orElse(KeyField.none()); final DataSource dataSource; @@ -959,7 +961,7 @@ private void givenDataSourceWithSchema( private static Struct keyStruct(final String rowKey) { final Struct key = new Struct(SCHEMA.keyConnectSchema()); - key.put("ROWKEY", rowKey); + key.put(Iterables.getOnlyElement(SCHEMA.key()).name().text(), rowKey); return key; } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java index 861b6af91b13..05105d4e5f17 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java @@ -537,6 +537,7 @@ private PersistentQueryMetadata executeQuery( private static Struct asKeyStruct(final String rowKey, final PhysicalSchema physicalSchema) { final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); + // TOdo(ac): key.put(SchemaUtil.ROWKEY_NAME.text(), rowKey); return key; } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java b/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java index 29ccd0601432..0966035b5b4a 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java @@ -352,7 +352,7 @@ public void shouldThrowIfLeftTableNotJoiningOnTableKey() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage("Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE key `ID` instead of COL0"); + + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0"); // When: execute("CREATE TABLE t1 AS " @@ -369,7 +369,7 @@ public void shouldThrowIfRightTableNotJoiningOnTableKey() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage("Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE key `ID` instead of COL0"); + + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0"); // When: execute("CREATE TABLE t1 AS " diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java index e7d93e9aa27f..0897f44d743b 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java @@ -54,7 +54,6 @@ import io.confluent.ksql.structured.SchemaKStream; import io.confluent.ksql.structured.SchemaKTable; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.SchemaUtil; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -75,8 +74,7 @@ @RunWith(MockitoJUnitRunner.class) public class DataSourceNodeTest { - private static final ColumnName TIMESTAMP_FIELD - = ColumnName.of("timestamp"); + private static final ColumnName TIMESTAMP_FIELD = ColumnName.of("timestamp"); private static final PlanNodeId PLAN_NODE_ID = new PlanNodeId("0"); private static final SourceName SOURCE_NAME = SourceName.of("datasource"); @@ -89,7 +87,7 @@ public class DataSourceNodeTest { private static final ColumnName FIELD3 = ColumnName.of("field3"); private static final LogicalSchema REAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.INTEGER) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(FIELD1, SqlTypes.INTEGER) .valueColumn(FIELD2, SqlTypes.STRING) .valueColumn(FIELD3, SqlTypes.STRING) diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java index 15ae6f55cbc5..c504af35da29 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java @@ -28,7 +28,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; @@ -61,7 +60,6 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -90,13 +88,13 @@ public class JoinNodeTest { private static final LogicalSchema LEFT_SOURCE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("C0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("L1"), SqlTypes.STRING) .build(); private static final LogicalSchema RIGHT_SOURCE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("rightKey"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("C0"), SqlTypes.STRING) .valueColumn(ColumnName.of("R1"), SqlTypes.BIGINT) .build(); @@ -646,15 +644,15 @@ public void shouldHaveFullyQualifiedJoinSchema() { // When: assertThat(joinNode.getSchema(), is(LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "C0"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "L1"), SqlTypes.STRING) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "ROWTIME"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "ROWKEY"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "C0"), SqlTypes.STRING) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "R1"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "ROWTIME"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "ROWKEY"), SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_C0"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_L1"), SqlTypes.STRING) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_ROWTIME"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_leftKey"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_C0"), SqlTypes.STRING) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_R1"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_ROWTIME"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_rightKey"), SqlTypes.BIGINT) .build() )); } @@ -696,7 +694,7 @@ public void shouldReturnCorrectSchema() { // Then: assertThat(joinNode.getSchema(), is(LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) .valueColumns(LEFT_NODE_SCHEMA.value()) .valueColumns(RIGHT_NODE_SCHEMA.value()) .build())); @@ -779,25 +777,6 @@ private static Optional getColumn(final LogicalSchema schema, final Pred .findFirst(); } - private static ColumnName getNonKeyColumn( - final LogicalSchema schema, - final SourceName alias, - final ColumnName keyName - ) { - final ImmutableList blackList = ImmutableList.of( - SchemaUtil.ROWKEY_NAME, - SchemaUtil.ROWTIME_NAME, - keyName - ); - - final Column column = - getColumn(schema, s -> !blackList.contains(s)) - .orElseThrow(AssertionError::new); - - final Column field = schema.findValueColumn(column.name()).get(); - return field.name(); - } - private static void setUpSource( final DataSourceNode node, final ValueFormat valueFormat, diff --git a/ksql-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java b/ksql-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java index ee0100d42946..e23285298567 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/util/ItemDataProvider.java @@ -26,6 +26,7 @@ import io.confluent.ksql.serde.SerdeOption; import java.util.Map; +// Todo(ac): next: update data providers & demos to use custom key names, not WITH(KEY) public class ItemDataProvider extends TestDataProvider { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/RowGenerator.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/RowGenerator.java index 4fb921cbca8c..1f7c2578386f 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/RowGenerator.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/RowGenerator.java @@ -49,7 +49,7 @@ public class RowGenerator { private final Generator generator; private final AvroData avroData; private final SessionManager sessionManager = new SessionManager(); - private final String keyFieldName; + private final String keyFieldName; // Todo(ac): future: remove. private final ConnectSchema keySchema; private final ConnectSchema valueSchema; private final int keyFieldIndex; @@ -141,6 +141,7 @@ public Pair generateRow() { } } + // Todo(ac): next final Struct keyStruct = new Struct(keySchema); keyStruct.put(SchemaUtil.ROWKEY_NAME.text(), row.get(keyFieldIndex)); @@ -280,6 +281,7 @@ private static LogicalSchema buildLogicalSchema( final Builder schemaBuilder = LogicalSchema.builder(); final ConnectToSqlTypeConverter converter = SchemaConverters.connectToSqlConverter(); + // Todo(ac): next schemaBuilder .keyColumn(SchemaUtil.ROWKEY_NAME, converter.toSqlType(keyField.schema())); diff --git a/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java b/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java index 2282fdce9559..4decad802764 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java @@ -20,7 +20,6 @@ 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.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.serde.WindowInfo; @@ -108,16 +107,16 @@ private static void validate(final LogicalSchema schema, final Optional" + "schema": "ID BIGINT KEY, FOO STRING, KSQL_COL_1 BIGINT" } ``` @@ -303,8 +302,7 @@ Each source can define the following attributes: |-------------|:------------| | name | (Required) The name of the source. | | type | (Required) Specifies if the source is a STREAM or TABLE. | -| keyField | (Optional) Specifies the keyField for the source. If present, but set to `null`, the name of the key field is expected to not be set. If not supplied, the name of the key field will not be checked. | -| valueSchema | (Optional) Specifies the value SQL schema for the source. | +| schema | (Optional) Specifies the SQL schema for the source. | #### Topics A post condition can define a check against the set of topics the case creates diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json new file mode 100644 index 000000000000..c50ef3113a0f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (KEY STRING KEY, ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`KEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n INPUT.KEY KEY2\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`KEY` STRING KEY, `ID` BIGINT, `KEY2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`KEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "KEY AS KEY2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent6357789129346422021", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "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.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "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.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/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json new file mode 100644 index 000000000000..5242b6561f65 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json @@ -0,0 +1,23 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583166643536, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : "a", + "value" : { + "id" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : { + "ID" : 1, + "KEY2" : "a" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json new file mode 100644 index 000000000000..031a3f1fe847 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (OTHER DOUBLE KEY, ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`OTHER` DOUBLE KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n INPUT.OTHER KEY\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`OTHER` DOUBLE KEY, `ID` BIGINT, `KEY` DOUBLE", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`OTHER` DOUBLE KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "OTHER AS KEY" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent6357789129346422021", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "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.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "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.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/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json new file mode 100644 index 000000000000..54cc810810a1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json @@ -0,0 +1,23 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583166643478, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : 3.0, + "value" : { + "id" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 3.0, + "value" : { + "ID" : 1, + "KEY" : 3.0 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json b/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json index b659972e0e9f..118908c9e025 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json @@ -574,10 +574,27 @@ "CREATE STREAM OUTPUT AS SELECT ROWKEY FROM INPUT;" ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Reserved column name in select: `ROWKEY`. Please remove or alias the column." } }, + { + "name": "non-join should reject KEY column name in projection", + "comments": [ + "changes to the ksql query semantics are required to allow this.", + "At the moment, the key schema passes through the select un-changed.", + "which means the key column in the projection is added to the value schema", + "but the name of the column clashes with the key column, resulting in an error" + ], + "statements": [ + "CREATE STREAM INPUT (K INT KEY, F0 INT) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT K FROM INPUT;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Value column name 'K' clashes with key column name." + } + }, { "name": "non-join should reject WINDOWSTART in projection", "statements": [ @@ -629,10 +646,28 @@ "CREATE STREAM OUTPUT as SELECT l.ROWKEY AS ROWKEY, f0, f1 FROM left_stream l join right_stream r WITHIN 1 seconds ON l.rowkey = r.rowkey;" ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Reserved column name in select: `ROWKEY`. Please remove or alias the column." } }, + { + "name": "join should reject KEY column name in projection", + "comments": [ + "changes to the ksql query semantics are required to allow this.", + "At the moment, the key schema passes through the select un-changed.", + "which means the key column in the projection is added to the value schema", + "but the name of the column clashes with the key column, resulting in an error" + ], + "statements": [ + "CREATE STREAM LEFT_STREAM (K DOUBLE KEY, F0 INT) WITH (kafka_topic='left', value_format='JSON');", + "CREATE STREAM RIGHT_STREAM (K DOUBLE KEY, F1 INT) WITH (kafka_topic='right', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT l.K AS K, f0, f1 FROM left_stream l join right_stream r WITHIN 1 seconds ON l.k = r.k;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Value column name 'K' clashes with key column name." + } + }, { "name": "join should reject WINDOWSTART in projection", "statements": [ diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json b/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json index 1eda66ebc2c1..c802ecad72a4 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json @@ -1457,7 +1457,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE key ROWKEY instead of ID" + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column ROWKEY instead of ID" } }, { @@ -1960,7 +1960,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE key ROWKEY instead of ID" + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column ROWKEY instead of ID" } }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json b/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json index c631d12c8b3e..a34e4b6b1c9c 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json @@ -831,7 +831,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "The KEY field (FOO) identified in the WITH clause is of a different type to the actual key column.\nEither change the type of the KEY field to match ROWKEY, or explicitly set ROWKEY to the type of the KEY field by adding 'ROWKEY INTEGER KEY' in the schema.\nKEY field type: INTEGER\nROWKEY type: STRING" + "message": "The KEY field (FOO) identified in the WITH clause is of a different type to the actual key column.\nUse of the KEY field is deprecated. Remove the KEY field from the WITH clause and specify the name of the key column by adding 'FOO INTEGER KEY' to the schema.\nKEY field type: INTEGER\nkey column type: STRING" } }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json b/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json index 2acb23c20dbe..73bf4738945f 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json @@ -319,22 +319,48 @@ { "name": "explicit key field named other than ROWKEY", "statements": [ - "CREATE STREAM INPUT (OTHER STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');" + "CREATE STREAM INPUT (OTHER DOUBLE KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT ID, OTHER as KEY FROM INPUT;" ], - "expectedException": { - "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "'OTHER' is an invalid KEY column name. KSQL currently only supports KEY columns named ROWKEY." + "inputs": [ + {"topic": "input", "key": 3.0, "value": {"id": 1}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 3.0, "value": {"ID": 1, "KEY": 3.0}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "OTHER DOUBLE KEY, ID BIGINT, KEY DOUBLE" + } + ] } }, { "name": "KEY key field name", "comment": "tests that `KEY` is allowed as a KEY column name", "statements": [ - "CREATE STREAM INPUT (KEY STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');" + "CREATE STREAM INPUT (KEY STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT ID, KEY as KEY2 FROM INPUT;" ], - "expectedException": { - "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "'KEY' is an invalid KEY column name. KSQL currently only supports KEY columns named ROWKEY." + "inputs": [ + {"topic": "input", "key": "a", "value": {"id": 1}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "a", "value": {"ID": 1, "KEY2": "a"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "KEY STRING KEY, ID BIGINT, KEY2 STRING" + } + ] } }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json b/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json index cd832344495e..ac4a72dfa126 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json @@ -4,7 +4,17 @@ ], "tests": [ { - "name": "should fail if ROWKEY used as column name", + "name": "should fail if ROWKEY used as key column name", + "statements": [ + "CREATE STREAM INPUT (ROWKEY int) WITH (kafka_topic='test', value_format='JSON');" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "'ROWKEY' is a reserved column name. It can only be used for KEY columns." + } + }, + { + "name": "should fail if ROWKEY used as value column name", "statements": [ "CREATE STREAM INPUT (x int) WITH (kafka_topic='test', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT x AS rowkey FROM INPUT;" diff --git a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json index 8765665c1b53..6d64d41c1cbb 100644 --- a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json +++ b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json @@ -6,8 +6,8 @@ { "name": "explicitly supply all column values", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO TEST (ROWTIME, ROWKEY, ID) VALUES (1234, 'key', 10);" + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (ROWTIME, K, ID) VALUES (1234, 'key', 10);" ], "inputs": [ ], @@ -21,8 +21,8 @@ { "name": "explicitly supply values out of order", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO TEST (ID, ROWTIME, ROWKEY) VALUES (10, 1234, 'key');" + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (ID, ROWTIME, K) VALUES (10, 1234, 'key');" ], "inputs": [ ], @@ -33,8 +33,8 @@ { "name": "explicitly supply default set of column values", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO TEST (ROWKEY, ID) VALUES ('key', 10);" + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K, ID) VALUES ('key', 10);" ], "inputs": [ ], @@ -45,7 +45,7 @@ { "name": "implicitly supply default set of column values", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST VALUES ('key', 10);" ], "inputs": [ @@ -57,8 +57,8 @@ { "name": "should insert nulls for any fields not provided", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO TEST (ROWKEY) VALUES ('10');" + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K) VALUES ('10');" ], "inputs": [ ], @@ -81,7 +81,7 @@ { "name": "rowkey should be set when stream has int key and only key specified in insert", "statements": [ - "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (10);" ], "inputs": [ @@ -105,7 +105,7 @@ { "name": "rowkey should be set when stream has double key and only key specified in insert", "statements": [ - "CREATE STREAM TEST (ROWKEY DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "CREATE STREAM TEST (K DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (1.23);" ], "inputs": [ @@ -117,7 +117,7 @@ { "name": "rowkey should be set when stream has bigint key and only key specified in insert", "statements": [ - "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (10);" ], "inputs": [ @@ -129,8 +129,8 @@ { "name": "keyfield should be set when stream has string key and only rowkey specified in insert", "statements": [ - "CREATE STREAM TEST (ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY) VALUES ('10');" + "CREATE STREAM TEST (K STRING KEY, ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K) VALUES ('10');" ], "inputs": [ ], @@ -141,8 +141,8 @@ { "name": "rowkey and key should match when stream has int key", "statements": [ - "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 10);" + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 10);" ], "inputs": [ ], @@ -153,8 +153,8 @@ { "name": "rowkey and key should match when stream has String key", "statements": [ - "CREATE STREAM TEST (ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY, ID) VALUES ('10', '10');" + "CREATE STREAM TEST (K STRING KEY, ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES ('10', '10');" ], "inputs": [ ], @@ -165,8 +165,8 @@ { "name": "rowkey and key should match when stream has double key", "statements": [ - "CREATE STREAM TEST (ROWKEY DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY, ID) VALUES (1.23, 1.23);" + "CREATE STREAM TEST (K DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (1.23, 1.23);" ], "inputs": [ ], @@ -177,8 +177,8 @@ { "name": "rowkey and key should match when stream has bigint key", "statements": [ - "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 10);" + "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 10);" ], "inputs": [ ], @@ -189,24 +189,24 @@ { "name": "should fail on mismatch between explicit columns and value counts", "statements": [ - "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO TEST (ROWKEY, ID) VALUES ('10');" + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K, ID) VALUES ('10');" ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "Failed to prepare statement: Expected number columns and values to match: [ROWKEY, ID], ['10']", + "message": "Failed to prepare statement: Expected number columns and values to match: [K, ID], ['10']", "status": 400 } }, { "name": "should fail on mismatch between rowkey and key values when stream has key", "statements": [ - "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", - "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 5);" + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 5);" ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "Failed to insert values into 'TEST'. Expected ROWKEY and ID to match but got 10 and 5 respectively.", + "message": "Failed to insert values into 'TEST'. Expected K and ID to match but got 10 and 5 respectively.", "status": 400 } }, @@ -261,8 +261,8 @@ { "name": "should handle quoted identifiers", "statements": [ - "CREATE STREAM `test` (`id!` INT) WITH (kafka_topic='test_topic', value_format='JSON');", - "INSERT INTO `test` (ROWTIME, ROWKEY, `id!`) VALUES (1234, 'key', 10);" + "CREATE STREAM `test` (`@key` STRING KEY, `id!` INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO `test` (ROWTIME, `@key`, `id!`) VALUES (1234, 'key', 10);" ], "inputs": [ ], @@ -276,7 +276,7 @@ { "name": "should handle struct expressions", "statements": [ - "CREATE STREAM TEST (val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2.1, `bar` := ARRAY['bar']));" ], "inputs": [ @@ -288,7 +288,7 @@ { "name": "should handle struct coercion", "statements": [ - "CREATE STREAM TEST (val STRUCT, baz DOUBLE>) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, val STRUCT, baz DOUBLE>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2, BAR := ARRAY[2], BAZ := 2));" ], "inputs": [ @@ -300,7 +300,7 @@ { "name": "should handle empty struct expressions", "statements": [ - "CREATE STREAM TEST (val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT());" ], "inputs": [ @@ -308,6 +308,18 @@ "outputs": [ {"topic": "test_topic", "key": null, "value": {"VAL": {"FOO": null, "bar": null}}} ] + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM TEST (`Key` STRING KEY, `Value` INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (`Key`, `Value`) VALUES ('key', 10);" + ], + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "key", "value": {"Value": 10}} + ] } ] } \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json index c33ae56576b0..0723ea83030a 100644 --- a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json +++ b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json @@ -105,7 +105,7 @@ ] }, { - "name": "non-windowed lookup on wrong type type", + "name": "non-windowed lookup on wrong type", "statements": [ "CREATE STREAM INPUT (ROWKEY INT KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", @@ -113,7 +113,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "'10' can not be converted to the type of column ROWKEY: INTEGER", + "message": "'10' can not be converted to the type of the key column: ROWKEY INTEGER KEY", "status": 400 } }, @@ -222,7 +222,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "'10' can not be converted to the type of column ROWKEY: INTEGER", + "message": "'10' can not be converted to the type of the key column: ROWKEY INTEGER KEY", "status": 400 } }, @@ -901,7 +901,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "Multiple bounds on ROWKEY", + "message": "Multiple bounds on key column", "status": 400 } }, @@ -914,7 +914,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "ROWKEY bound must currently be '='.", + "message": "Bound on 'ROWKEY' must currently be '='.", "status": 400 } }, @@ -927,7 +927,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "WHERE clause on unsupported field: COUNT", + "message": "WHERE clause on unsupported column: COUNT", "status": 400 } }, @@ -1015,6 +1015,26 @@ {"row":{"columns":[12345, 10.1, 12000, 13000]}} ]} ] + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM INPUT (`Key` STRING KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS `Value` FROM INPUT GROUP BY `Key`;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10';" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12365, "key": "10", "value": {}} + ], + "responses": [ + {"admin": {"@type": "currentStatus"}}, + {"admin": {"@type": "currentStatus"}}, + {"query": [ + {"header":{"schema":"`ROWKEY` STRING KEY, `ROWTIME` BIGINT, `Value` BIGINT"}}, + {"row":{"columns":["10", 12365, 1]}} + ]} + ] } ] } \ No newline at end of file diff --git a/ksql-metastore/src/main/java/io/confluent/ksql/metastore/model/StructuredDataSource.java b/ksql-metastore/src/main/java/io/confluent/ksql/metastore/model/StructuredDataSource.java index a4b27f4f2081..3bb892e7a34c 100644 --- a/ksql-metastore/src/main/java/io/confluent/ksql/metastore/model/StructuredDataSource.java +++ b/ksql-metastore/src/main/java/io/confluent/ksql/metastore/model/StructuredDataSource.java @@ -21,12 +21,15 @@ import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; 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.serde.SerdeOption; import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; @Immutable abstract class StructuredDataSource implements DataSource { @@ -66,6 +69,14 @@ abstract class StructuredDataSource implements DataSource { if (schema.valueContainsAny(SchemaUtil.systemColumnNames())) { throw new IllegalArgumentException("Schema contains system columns in value schema"); } + + final Set keyNames = schema.key().stream() + .map(Column::name) + .collect(Collectors.toSet()); + + if (schema.valueContainsAny(keyNames)) { + throw new IllegalArgumentException("Schema contains duplicate column names"); + } } @Override diff --git a/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java b/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java index 3eaaecaf59c0..4f0dff1824b8 100644 --- a/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java +++ b/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java @@ -69,10 +69,11 @@ public void shouldThrowIfSchemaContainsRowTime() { } @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfSchemaContainsRowKey() { + public void shouldThrowOnDuplicateColumnNames() { // Given: final LogicalSchema schema = LogicalSchema.builder() - .valueColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("dup"), SqlTypes.INTEGER) + .valueColumn(ColumnName.of("dup"), SqlTypes.STRING) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); diff --git a/ksql-metastore/src/test/java/io/confluent/ksql/util/MetaStoreFixture.java b/ksql-metastore/src/test/java/io/confluent/ksql/util/MetaStoreFixture.java index 9c5594f673cb..4643b4bc235e 100644 --- a/ksql-metastore/src/test/java/io/confluent/ksql/util/MetaStoreFixture.java +++ b/ksql-metastore/src/test/java/io/confluent/ksql/util/MetaStoreFixture.java @@ -43,6 +43,7 @@ public static MutableMetaStore getNewMetaStore(final FunctionRegistry functionRe return getNewMetaStore(functionRegistry, ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name()))); } + // Todo(ac): Switch one stream and one table to have custom key name public static MutableMetaStore getNewMetaStore( final FunctionRegistry functionRegistry, final ValueFormat valueFormat diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java index 99f15b35494b..48d96c5ce113 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java @@ -16,14 +16,15 @@ package io.confluent.ksql.parser.tree; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.tree.TableElement.Namespace; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.LogicalSchema.Builder; import io.confluent.ksql.schema.ksql.types.SqlType; +import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.SchemaUtil; import java.util.Iterator; import java.util.List; import java.util.Map.Entry; @@ -79,15 +80,23 @@ public String toString() { /** * @param withImplicitColumns controls if schema has implicit columns such as ROWTIME or ROWKEY. * @return the logical schema. - */ + */ // Todo(ac): Ditch param? public LogicalSchema toLogicalSchema(final boolean withImplicitColumns) { - if (Iterables.isEmpty(this)) { + if (elements.isEmpty()) { throw new KsqlException("No columns supplied."); } - final Builder builder = withImplicitColumns - ? LogicalSchema.builder() - : LogicalSchema.builder().noImplicitColumns(); + final Builder builder = LogicalSchema.builder().noImplicitColumns(); + + if (withImplicitColumns) { + builder.withRowTime(); + + // Todo(ac): allow no key to mean no key! + final boolean noKey = elements.stream().noneMatch(e -> e.getNamespace() == Namespace.KEY); + if (noKey) { + builder.keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING); + } + } for (final TableElement tableElement : this) { final ColumnName fieldName = tableElement.getName(); diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java index b96e864a2878..95e5d21b1593 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java @@ -92,7 +92,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.math.BigDecimal; import java.util.List; import java.util.Objects; @@ -136,7 +135,7 @@ public class KsqlParserTest { .build(); private static final LogicalSchema ORDERS_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java index c07e8aef95a1..4d6eb9984ba8 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java @@ -72,7 +72,7 @@ public void shouldParseValidSchema() { } @Test - public void shouldParseValidSchemaWithKeyField() { + public void shouldParseValidSchemaWithRowKeyField() { // Given: final String schema = "ROWKEY STRING KEY, bar INT"; @@ -86,6 +86,21 @@ public void shouldParseValidSchemaWithKeyField() { )); } + @Test + public void shouldParseValidSchemaWithKeyField() { + // Given: + final String schema = "K STRING KEY, bar INT"; + + // When: + final TableElements elements = parser.parse(schema); + + // Then: + assertThat(elements, contains( + new TableElement(Namespace.KEY, ColumnName.of("K"), new Type(SqlTypes.STRING)), + new TableElement(Namespace.VALUE, BAR, new Type(SqlTypes.INTEGER)) + )); + } + @Test public void shouldParseQuotedSchema() { // Given: diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java index 432944c0c9fd..e99f9a3d1bcf 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java @@ -67,7 +67,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import java.util.concurrent.TimeUnit; import org.junit.Before; @@ -109,7 +108,7 @@ public class SqlFormatterTest { .build(); private static final LogicalSchema ITEM_INFO_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("NAME"), SqlTypes.STRING) .valueColumn(ColumnName.of("CATEGORY"), categorySchema) @@ -120,7 +119,7 @@ public class SqlFormatterTest { .build(); private static final LogicalSchema ORDERS_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K1"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) @@ -140,7 +139,7 @@ CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral("topic_test")) ); private static final TableElements ELEMENTS_WITH_KEY = TableElements.of( - new TableElement(Namespace.KEY, ColumnName.of("ROWKEY"), new Type(SqlTypes.STRING)), + new TableElement(Namespace.KEY, ColumnName.of("k3"), new Type(SqlTypes.STRING)), new TableElement(Namespace.VALUE, ColumnName.of("Foo"), new Type(SqlTypes.STRING)) ); @@ -168,7 +167,7 @@ public void setUp() { ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())) ); - final KsqlStream ksqlStreamOrders = new KsqlStream<>( + final KsqlStream ksqlStreamOrders = new KsqlStream<>( "sqlexpression", SourceName.of("ADDRESS"), ORDERS_SCHEMA, @@ -226,7 +225,7 @@ public void shouldFormatCreateStreamStatementWithExplicitKey() { final String sql = SqlFormatter.formatSql(createStream); // Then: - assertThat(sql, is("CREATE STREAM TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE STREAM TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', VALUE_FORMAT='JSON');")); } @@ -267,7 +266,7 @@ public void shouldFormatCreateTableStatementWithExplicitTimestamp() { final String sql = SqlFormatter.formatSql(createTable); // Then: - assertThat(sql, is("CREATE TABLE TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE TABLE TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', " + "TIMESTAMP='Foo', TIMESTAMP_FORMAT='%s', VALUE_FORMAT='JSON');")); } @@ -285,7 +284,7 @@ public void shouldFormatCreateTableStatementWithExplicitKey() { final String sql = SqlFormatter.formatSql(createTable); // Then: - assertThat(sql, is("CREATE TABLE TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE TABLE TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', VALUE_FORMAT='JSON');")); } diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java index 67a4b368111f..5c80a52b8e63 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java @@ -25,7 +25,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.NodeLocation; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import org.junit.Rule; import org.junit.Test; @@ -41,6 +40,7 @@ public class TableElementTest { @Rule public final ExpectedException expectedException = ExpectedException.none(); + @SuppressWarnings("UnstableApiUsage") @Test public void shouldImplementEquals() { new EqualsTester() @@ -55,7 +55,7 @@ public void shouldImplementEquals() { new TableElement(VALUE, NAME, new Type(SqlTypes.INTEGER)) ) .addEqualityGroup( - new TableElement(KEY, SchemaUtil.ROWKEY_NAME, new Type(SqlTypes.STRING)) + new TableElement(KEY, NAME, new Type(SqlTypes.STRING)) ) .testEquals(); } diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java index d8d49afd1c2d..0840801ba750 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java @@ -222,6 +222,26 @@ public void shouldBuildLogicalSchemaWithImplicits() { )); } + @Test + public void shouldBuildLogicalSchemaWithImplicitsAndExplicitKey() { + // Given: + final TableElements tableElements = TableElements.of( + tableElement(VALUE, "v0", INT_TYPE), + tableElement(KEY, "k0", INT_TYPE) + ); + + // When: + final LogicalSchema schema = tableElements.toLogicalSchema(true); + + // Then: + assertThat(schema, is(LogicalSchema.builder() + .withRowTime() + .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) + .build() + )); + } + @Test public void shouldBuildLogicalSchemaWithOutImplicits() { // Given: diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java index d107b83a5652..117c91f47e15 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java @@ -62,6 +62,7 @@ import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.AllColumns; import io.confluent.ksql.parser.tree.Query; @@ -76,6 +77,7 @@ import io.confluent.ksql.rest.entity.TableRowsEntity; import io.confluent.ksql.rest.entity.TableRowsEntityFactory; import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -104,6 +106,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,6 +140,7 @@ public PullQueryExecutor( Objects.requireNonNull(routingFilterFactory, "routingFilterFactory"); } + @SuppressWarnings("unused") // Needs to match validator API. public static void validate( final ConfiguredStatement statement, final SessionProperties sessionProperties, @@ -181,10 +185,10 @@ public TableRowsEntity execute( .getMaterialization(queryId, contextStacker) .orElseThrow(() -> notMaterializedException(getSourceName(analysis))); - final Struct rowKey = asKeyStruct(whereInfo.rowkey, query.getPhysicalSchema()); + final Struct key = asKeyStruct(whereInfo.keyBound, query.getPhysicalSchema()); final PullQueryContext pullQueryContext = new PullQueryContext( - rowKey, + key, mat, analysis, whereInfo, @@ -219,7 +223,7 @@ private TableRowsEntity handlePullQuery( // Get active and standby nodes for this key final Locator locator = pullQueryContext.mat.locator(); final List filteredAndOrderedNodes = locator.locate( - pullQueryContext.rowKey, + pullQueryContext.key, routingOptions, routingFilterFactory ); @@ -264,7 +268,7 @@ private TableRowsEntity routeQuery( } @VisibleForTesting - TableRowsEntity queryRowsLocally( + static TableRowsEntity queryRowsLocally( final ConfiguredStatement statement, final KsqlExecutionContext executionContext, final PullQueryContext pullQueryContext @@ -274,12 +278,12 @@ TableRowsEntity queryRowsLocally( final Range windowStart = pullQueryContext.whereInfo.windowStartBounds.get(); final List rows = pullQueryContext.mat.windowed() - .get(pullQueryContext.rowKey, windowStart); + .get(pullQueryContext.key, windowStart); result = new Result(pullQueryContext.mat.schema(), rows); } else { final List rows = pullQueryContext.mat.nonWindowed() - .get(pullQueryContext.rowKey) + .get(pullQueryContext.key) .map(ImmutableList::of) .orElse(ImmutableList.of()); @@ -371,11 +375,11 @@ TableRowsEntity forwardTo( ); } - private QueryId uniqueQueryId() { + private static QueryId uniqueQueryId() { return new QueryId("query_" + System.currentTimeMillis()); } - private ImmutableAnalysis analyze( + private static ImmutableAnalysis analyze( final ConfiguredStatement statement, final KsqlExecutionContext executionContext ) { @@ -389,7 +393,8 @@ private ImmutableAnalysis analyze( } private static final class PullQueryContext { - private final Struct rowKey; + + private final Struct key; private final Materialization mat; private final ImmutableAnalysis analysis; private final WhereInfo whereInfo; @@ -397,14 +402,14 @@ private static final class PullQueryContext { private final QueryContext.Stacker contextStacker; private PullQueryContext( - final Struct rowKey, + final Struct key, final Materialization mat, final ImmutableAnalysis analysis, final WhereInfo whereInfo, final QueryId queryId, final QueryContext.Stacker contextStacker ) { - this.rowKey = Objects.requireNonNull(rowKey, "rowkey"); + this.key = Objects.requireNonNull(key, "key"); this.mat = Objects.requireNonNull(mat, "materialization"); this.analysis = Objects.requireNonNull(analysis, "analysis"); this.whereInfo = Objects.requireNonNull(whereInfo, "whereInfo"); @@ -412,8 +417,8 @@ private PullQueryContext( this.contextStacker = Objects.requireNonNull(contextStacker, "contextStacker"); } - public Struct getRowKey() { - return rowKey; + public Struct getKey() { + return key; } public Materialization getMat() { @@ -439,14 +444,14 @@ public QueryContext.Stacker getContextStacker() { private static final class WhereInfo { - private final Object rowkey; + private final Object keyBound; private final Optional> windowStartBounds; private WhereInfo( - final Object rowkey, + final Object keyBound, final Optional> windowStartBounds ) { - this.rowkey = rowkey; + this.keyBound = keyBound; this.windowStartBounds = windowStartBounds; } } @@ -465,7 +470,7 @@ private Result( } } - private WhereInfo extractWhereInfo( + private static WhereInfo extractWhereInfo( final ImmutableAnalysis analysis, final PersistentQueryMetadata query ) { @@ -474,15 +479,16 @@ private WhereInfo extractWhereInfo( final Expression where = analysis.getWhereExpression() .orElseThrow(() -> invalidWhereClauseException("Missing WHERE clause", windowed)); - final Map> comparisons = extractComparisons(where); + final Map> comparisons = + extractComparisons(where, query); - final List rowKeyComparison = comparisons.get(ComparisonTarget.ROWKEY); - if (rowKeyComparison == null) { - throw invalidWhereClauseException("WHERE clause missing ROWKEY", windowed); + final List keyComparison = comparisons.get(ComparisonTarget.KEYCOL); + if (keyComparison == null) { + throw invalidWhereClauseException("WHERE clause missing key column", windowed); } - final Object rowKey = extractRowKeyWhereClause( - rowKeyComparison, + final Object key = extractKeyWhereClause( + keyComparison, windowed, query.getLogicalSchema() ); @@ -492,7 +498,7 @@ private WhereInfo extractWhereInfo( throw invalidWhereClauseException("Unsupported WHERE clause", false); } - return new WhereInfo(rowKey, Optional.empty()); + return new WhereInfo(key, Optional.empty()); } final Optional> windowBoundsComparison = @@ -500,30 +506,32 @@ private WhereInfo extractWhereInfo( final Range windowStart = extractWhereClauseWindowBounds(windowBoundsComparison); - return new WhereInfo(rowKey, Optional.of(windowStart)); + return new WhereInfo(key, Optional.of(windowStart)); } - private Object extractRowKeyWhereClause( + private static Object extractKeyWhereClause( final List comparisons, final boolean windowed, final LogicalSchema schema ) { if (comparisons.size() != 1) { - throw invalidWhereClauseException("Multiple bounds on ROWKEY", windowed); + throw invalidWhereClauseException("Multiple bounds on key column", windowed); } final ComparisonExpression comparison = comparisons.get(0); if (comparison.getType() != Type.EQUAL) { - throw invalidWhereClauseException("ROWKEY bound must currently be '='", windowed); + final ColumnName keyColumn = Iterables.getOnlyElement(schema.key()).name(); + throw invalidWhereClauseException("Bound on '" + keyColumn.text() + + "' must currently be '='", windowed); } final Expression other = getNonColumnRefSide(comparison); final Object right = ((Literal) other).getValue(); - return coerceRowKey(schema, right, windowed); + return coerceKey(schema, right, windowed); } - private Object coerceRowKey( + private static Object coerceKey( final LogicalSchema schema, final Object right, final boolean windowed @@ -532,14 +540,14 @@ private Object coerceRowKey( throw invalidWhereClauseException("Only single KEY column supported", windowed); } - final SqlType sqlType = schema.key().get(0).type(); + final Column keyColumn = schema.key().get(0); - return DefaultSqlValueCoercer.INSTANCE.coerce(right, sqlType) + return DefaultSqlValueCoercer.INSTANCE.coerce(right, keyColumn.type()) .orElseThrow(() -> new KsqlException("'" + right + "' can not be converted " - + "to the type of column ROWKEY: " + sqlType)); + + "to the type of the key column: " + keyColumn.toString(FormatOptions.noEscape()))); } - private Range extractWhereClauseWindowBounds( + private static Range extractWhereClauseWindowBounds( final Optional> maybeComparisons ) { if (!maybeComparisons.isPresent()) { @@ -549,7 +557,7 @@ private Range extractWhereClauseWindowBounds( final List comparisons = maybeComparisons.get(); final Map> byType = comparisons.stream() - .collect(Collectors.groupingBy(this::getSimplifiedBoundType)); + .collect(Collectors.groupingBy(PullQueryExecutor::getSimplifiedBoundType)); final SetView unsupported = Sets.difference(byType.keySet(), VALID_WINDOW_BOUNDS_TYPES); if (!unsupported.isEmpty()) { @@ -596,7 +604,7 @@ private Range extractWhereClauseWindowBounds( return extractWindowBound(lower, upper); } - private Type getSimplifiedBoundType(final ComparisonExpression comparison) { + private static Type getSimplifiedBoundType(final ComparisonExpression comparison) { final Type type = comparison.getType(); final boolean inverted = comparison.getRight() instanceof UnqualifiedColumnReferenceExp; @@ -612,7 +620,7 @@ private Type getSimplifiedBoundType(final ComparisonExpression comparison) { } } - private Range extractWindowBound( + private static Range extractWindowBound( final Optional lowerComparison, final Optional upperComparison ) { @@ -641,7 +649,7 @@ private Range extractWindowBound( return Range.range(lower, lowerType, upper, upperType); } - private BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { + private static BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { final boolean openBound = lowerComparison.getType() == Type.LESS_THAN || lowerComparison.getType() == Type.GREATER_THAN; @@ -650,13 +658,13 @@ private BoundType getRangeBoundType(final ComparisonExpression lowerComparison) : BoundType.CLOSED; } - private Expression getNonColumnRefSide(final ComparisonExpression comparison) { + private static Expression getNonColumnRefSide(final ComparisonExpression comparison) { return comparison.getRight() instanceof UnqualifiedColumnReferenceExp ? comparison.getLeft() : comparison.getRight(); } - private Instant asInstant(final Expression other) { + private static Instant asInstant(final Expression other) { if (other instanceof IntegerLiteral) { return Instant.ofEpochMilli(((IntegerLiteral) other).getValue()); } @@ -684,16 +692,17 @@ private Instant asInstant(final Expression other) { } private enum ComparisonTarget { - ROWKEY, + KEYCOL, WINDOWSTART } - private Map> extractComparisons( - final Expression exp + private static Map> extractComparisons( + final Expression exp, + final PersistentQueryMetadata query ) { if (exp instanceof ComparisonExpression) { final ComparisonExpression comparison = (ComparisonExpression) exp; - final ComparisonTarget target = extractWhereClauseTarget(comparison); + final ComparisonTarget target = extractWhereClauseTarget(comparison, query); return ImmutableMap.of(target, ImmutableList.of(comparison)); } @@ -704,10 +713,10 @@ private Map> extractComparisons( } final Map> left = - extractComparisons(binary.getLeft()); + extractComparisons(binary.getLeft(), query); final Map> right = - extractComparisons(binary.getRight()); + extractComparisons(binary.getRight(), query); return Stream .concat(left.entrySet().stream(), right.entrySet().stream()) @@ -719,7 +728,10 @@ private Map> extractComparisons( throw invalidWhereClauseException("Unsupported expression: " + exp, false); } - private ComparisonTarget extractWhereClauseTarget(final ComparisonExpression comparison) { + private static ComparisonTarget extractWhereClauseTarget( + final ComparisonExpression comparison, + final PersistentQueryMetadata query + ) { final UnqualifiedColumnReferenceExp column; if (comparison.getRight() instanceof UnqualifiedColumnReferenceExp) { column = (UnqualifiedColumnReferenceExp) comparison.getRight(); @@ -729,21 +741,28 @@ private ComparisonTarget extractWhereClauseTarget(final ComparisonExpression com throw invalidWhereClauseException("Invalid WHERE clause: " + comparison, false); } - final String fieldName = column.getReference().toString(FormatOptions.noEscape()); + final ColumnName columnName = column.getReference(); + if (columnName.equals(SchemaUtil.WINDOWSTART_NAME)) { + return ComparisonTarget.WINDOWSTART; + } - try { - return ComparisonTarget.valueOf(fieldName.toUpperCase()); - } catch (final Exception e) { - throw invalidWhereClauseException("WHERE clause on unsupported field: " + fieldName, false); + final ColumnName keyColumn = Iterables.getOnlyElement(query.getLogicalSchema().key()).name(); + if (columnName.equals(keyColumn)) { + return ComparisonTarget.KEYCOL; } + + throw invalidWhereClauseException( + "WHERE clause on unsupported column: " + columnName.text(), + false + ); } - private boolean isSelectStar(final Select select) { + private static boolean isSelectStar(final Select select) { final List selects = select.getSelectItems(); return selects.size() == 1 && selects.get(0) instanceof AllColumns; } - private List> handleSelects( + private static List> handleSelects( final Result input, final ConfiguredStatement statement, final KsqlExecutionContext executionContext, @@ -831,7 +850,7 @@ private List> handleSelects( return output.build(); } - private void validateProjection( + private static void validateProjection( final GenericRow fullRow, final LogicalSchema schema ) { @@ -845,7 +864,7 @@ private void validateProjection( } } - private LogicalSchema selectOutputSchema( + private static LogicalSchema selectOutputSchema( final Result input, final KsqlExecutionContext executionContext, final ImmutableAnalysis analysis, @@ -877,7 +896,7 @@ private LogicalSchema selectOutputSchema( return schemaBuilder.build(); } - private PersistentQueryMetadata findMaterializingQuery( + private static PersistentQueryMetadata findMaterializingQuery( final KsqlExecutionContext executionContext, final ImmutableAnalysis analysis ) { @@ -901,12 +920,12 @@ private PersistentQueryMetadata findMaterializingQuery( .orElseThrow(() -> new KsqlException("Materializing query has been stopped")); } - private SourceName getSourceName(final ImmutableAnalysis analysis) { + private static SourceName getSourceName(final ImmutableAnalysis analysis) { final DataSource source = analysis.getFromDataSources().get(0).getDataSource(); return source.getName(); } - private KsqlException notMaterializedException(final SourceName sourceTable) { + private static KsqlException notMaterializedException(final SourceName sourceTable) { return new KsqlException("'" + sourceTable.toString(FormatOptions.noEscape()) + "' is not materialized. " + PullQueryValidator.NEW_QUERY_SYNTAX_SHORT_HELP @@ -919,7 +938,7 @@ private KsqlException notMaterializedException(final SourceName sourceTable) { ); } - private KsqlException invalidWhereClauseException( + private static KsqlException invalidWhereClauseException( final String msg, final boolean windowed ) { @@ -943,19 +962,23 @@ private KsqlException invalidWhereClauseException( + System.lineSeparator() + "Pull queries require a WHERE clause that:" + System.lineSeparator() - + " - limits the query to a single ROWKEY, e.g. `SELECT * FROM X WHERE ROWKEY=Y;`." + + " - limits the query to a single key, e.g. `SELECT * FROM X WHERE >=Y;`." + additional ); } - private Struct asKeyStruct(final Object rowKey, final PhysicalSchema physicalSchema) { + private static Struct asKeyStruct(final Object keyValue, final PhysicalSchema physicalSchema) { + final Field keyField = Iterables + .getOnlyElement(physicalSchema.keySchema().ksqlSchema().fields()); + final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); - key.put(SchemaUtil.ROWKEY_NAME.text(), rowKey); + key.put(keyField, keyValue); return key; } private static final class ColumnReferenceRewriter extends VisitParentExpressionVisitor, Context> { + private ColumnReferenceRewriter() { super(Optional.empty()); } @@ -1008,7 +1031,7 @@ public long getOffsetLagAllowed() { public boolean skipForwardRequest() { return getForwardedFlag(KsqlRequestConfig.KSQL_REQUEST_QUERY_PULL_SKIP_FORWARDING); } - - } -} \ No newline at end of file +} + +// Todo(ac): Ensure this works with other KEY names. \ No newline at end of file diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java index b051fefb1717..341d54b222b5 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java @@ -308,7 +308,7 @@ public void shouldFailPullQueryWithNonKeyLookup() { String sql = "SELECT * from " + AGG_TABLE + " WHERE ROWTIME=12345;"; // Then: - shouldFailToExecuteQuery(sql, "WHERE clause on unsupported field: ROWTIME."); + shouldFailToExecuteQuery(sql, "WHERE clause on unsupported column: ROWTIME."); } @Test diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java index 1c7f5a6b1904..09934028351d 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java @@ -55,7 +55,7 @@ public final class GenericRowSerDe implements ValueSerdeFactory { *

Count covers the following additional columns: *

    *
  1. {@link SchemaUtil#ROWTIME_NAME}
  2. - *
  3. {@link SchemaUtil#ROWKEY_NAME}
  4. + *
  5. A single key column. (Which is the most common case)
  6. *
  7. {@link SchemaUtil#WINDOWSTART_NAME}
  8. *
  9. {@link SchemaUtil#WINDOWEND_NAME}
  10. *
diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java index 57c7a0d44eec..11500dcc0273 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java @@ -75,17 +75,19 @@ private static LogicalSchema multiExpressionSchema( private static LogicalSchema singleExpressionSchema( final LogicalSchema sourceSchema, - final SqlType rowKeyType + final SqlType keyType ) { - return buildSchemaWithKeyType(sourceSchema, rowKeyType); + return buildSchemaWithKeyType(sourceSchema, keyType); } private static LogicalSchema buildSchemaWithKeyType( final LogicalSchema sourceSchema, - final SqlType rowKeyType + final SqlType keyType ) { + + // Todo(ac): return LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, rowKeyType) + .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) .valueColumns(sourceSchema.value()) .build(); } @@ -140,11 +142,11 @@ private static final class SingleExpressionGrouper { } public Struct apply(final GenericRow row) { - final Object rowKey = processColumn(0, expression, row, logger); - if (rowKey == null) { + final Object key = processColumn(0, expression, row, logger); + if (key == null) { return null; } - return keyBuilder.build(rowKey); + return keyBuilder.build(key); } } @@ -168,21 +170,21 @@ private static final class MultiExpressionGrouper { } public Struct apply(final GenericRow row) { - final StringBuilder rowKey = new StringBuilder(); + final StringBuilder key = new StringBuilder(); for (int i = 0; i < expressions.size(); i++) { final Object result = processColumn(i, expressions.get(i), row, logger); if (result == null) { return null; } - if (rowKey.length() > 0) { - rowKey.append(GROUP_BY_VALUE_SEPARATOR); + if (key.length() > 0) { + key.append(GROUP_BY_VALUE_SEPARATOR); } - rowKey.append(result); + key.append(result); } - return keyBuilder.build(rowKey.toString()); + return keyBuilder.build(key.toString()); } } } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java index f8d754682252..17fa46af149c 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java @@ -98,7 +98,7 @@ public static KStreamHolder buildStream( source, queryBuilder, consumed, - nonWindowedRowKeyGenerator(source.getSourceSchema()) + nonWindowedKeyGenerator(source.getSourceSchema()) ); return new KStreamHolder<>( @@ -138,7 +138,7 @@ static KStreamHolder> buildWindowedStream( source, queryBuilder, consumed, - windowedRowKeyGenerator(source.getSourceSchema()) + windowedKeyGenerator(source.getSourceSchema()) ); return new KStreamHolder<>( @@ -184,7 +184,7 @@ public static KTableHolder buildTable( source, queryBuilder, consumed, - nonWindowedRowKeyGenerator(source.getSourceSchema()), + nonWindowedKeyGenerator(source.getSourceSchema()), materialized ); @@ -233,7 +233,7 @@ static KTableHolder> buildWindowedTable( source, queryBuilder, consumed, - windowedRowKeyGenerator(source.getSourceSchema()), + windowedKeyGenerator(source.getSourceSchema()), materialized ); @@ -275,27 +275,27 @@ private static KStream buildKStream( final SourceStep streamSource, final KsqlQueryBuilder queryBuilder, final Consumed consumed, - final Function> rowKeyGenerator + final Function> keyGenerator ) { final KStream stream = queryBuilder.getStreamsBuilder() .stream(streamSource.getTopicName(), consumed); return stream - .transformValues(new AddKeyAndTimestampColumns<>(rowKeyGenerator)); + .transformValues(new AddKeyAndTimestampColumns<>(keyGenerator)); } private static KTable buildKTable( final SourceStep streamSource, final KsqlQueryBuilder queryBuilder, final Consumed consumed, - final Function> rowKeyGenerator, + final Function> keyGenerator, final Materialized> materialized ) { final KTable table = queryBuilder.getStreamsBuilder() .table(streamSource.getTopicName(), consumed, materialized); return table - .transformValues(new AddKeyAndTimestampColumns<>(rowKeyGenerator)); + .transformValues(new AddKeyAndTimestampColumns<>(keyGenerator)); } private static TimestampExtractor timestampExtractor( @@ -352,7 +352,7 @@ private static String tableChangeLogOpName(final ExecutionStepPropertiesV1 props return StreamsUtil.buildOpName(stacker.push("Reduce").getQueryContext()); } - private static Function, Collection> windowedRowKeyGenerator( + private static Function, Collection> windowedKeyGenerator( final LogicalSchema schema ) { final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); @@ -368,7 +368,7 @@ private static Function, Collection> windowedRowKeyGenerator }; } - private static Function> nonWindowedRowKeyGenerator( + private static Function> nonWindowedKeyGenerator( final LogicalSchema schema ) { final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); @@ -384,10 +384,10 @@ private static Function> nonWindowedRowKeyGenerator( private static class AddKeyAndTimestampColumns implements ValueTransformerWithKeySupplier { - private final Function> rowKeyGenerator; + private final Function> keyGenerator; - AddKeyAndTimestampColumns(final Function> rowKeyGenerator) { - this.rowKeyGenerator = requireNonNull(rowKeyGenerator, "rowKeyGenerator"); + AddKeyAndTimestampColumns(final Function> keyGenerator) { + this.keyGenerator = requireNonNull(keyGenerator, "keyGenerator"); } @Override @@ -407,7 +407,7 @@ public GenericRow transform(final K key, final GenericRow row) { } final long timestamp = processorContext.timestamp(); - final Collection keyColumns = rowKeyGenerator.apply(key); + final Collection keyColumns = keyGenerator.apply(key); row.ensureAdditionalCapacity(1 + keyColumns.size()); row.append(timestamp); diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StepSchemaResolver.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StepSchemaResolver.java index 2c0bfaebfcb0..af83832a41d2 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StepSchemaResolver.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StepSchemaResolver.java @@ -217,6 +217,7 @@ private LogicalSchema handleSelectKey( final SqlType keyType = expressionTypeManager .getExpressionSqlType(step.getKeyExpression()); + // Todo(ac): return LogicalSchema.builder() .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) .valueColumns(sourceSchema.value()) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java index 469a4287ef46..475ecab77e51 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java @@ -9,7 +9,6 @@ import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; -import io.confluent.ksql.util.SchemaUtil; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -17,11 +16,13 @@ public class JoinParamsFactoryTest { private static final LogicalSchema LEFT_SCHEMA = LogicalSchema.builder() + .keyColumn(ColumnName.of("LK"), SqlTypes.STRING) .valueColumn(ColumnName.of("BLUE"), SqlTypes.STRING) .valueColumn(ColumnName.of("GREEN"), SqlTypes.INTEGER) .build(); private static final LogicalSchema RIGHT_SCHEMA = LogicalSchema.builder() + .keyColumn(ColumnName.of("RK"), SqlTypes.STRING) .valueColumn(ColumnName.of("RED"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.DOUBLE) .build(); @@ -38,7 +39,7 @@ public void shouldBuildCorrectSchema() { // Then: assertThat(joinParams.getSchema(), is(LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("LK"), SqlTypes.STRING) .valueColumn(ColumnName.of("BLUE"), SqlTypes.STRING) .valueColumn(ColumnName.of("GREEN"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("RED"), SqlTypes.BIGINT) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java index fa155739b6fa..3e8f2f6f3c86 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java @@ -54,7 +54,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.WindowInfo; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.SchemaUtil; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -94,20 +93,22 @@ public class SourceBuilderTest { + private static final ColumnName K0 = ColumnName.of("k0"); + private static final LogicalSchema SOURCE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(K0, SqlTypes.BIGINT) .valueColumn(ColumnName.of("field1"), SqlTypes.STRING) .valueColumn(ColumnName.of("field2"), SqlTypes.BIGINT) .build(); private static final Schema KEY_SCHEMA = SchemaBuilder.struct() - .field(SchemaUtil.ROWKEY_NAME.text(), Schema.OPTIONAL_FLOAT64_SCHEMA) + .field(K0.text(), Schema.OPTIONAL_FLOAT64_SCHEMA) .build(); private static final double A_KEY = 10.11; private static final Struct KEY = new Struct(KEY_SCHEMA) - .put(SchemaUtil.ROWKEY_NAME.text(), A_KEY); + .put(K0.text(), A_KEY); private static final LogicalSchema SCHEMA = SOURCE_SCHEMA .withMetaAndKeyColsInValue(false); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java index 53c742fe2386..bdb547f05b01 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java @@ -84,6 +84,7 @@ public class StepSchemaResolverTest { private static final KsqlConfig CONFIG = new KsqlConfig(Collections.emptyMap()); private static final LogicalSchema SCHEMA = LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("APPLE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BANANA"), SqlTypes.STRING) @@ -133,6 +134,7 @@ public void shouldResolveSchemaForStreamAggregate() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .build()) @@ -142,13 +144,13 @@ public void shouldResolveSchemaForStreamAggregate() { @Test public void shouldResolveSchemaForStreamWindowedAggregate() { // Given: - givenAggregateFunction("SUM", SqlTypes.BIGINT); + givenAggregateFunction("COUNT", SqlTypes.BIGINT); final StreamWindowedAggregate step = new StreamWindowedAggregate( PROPERTIES, groupedStreamSource, formats, ImmutableList.of(ColumnName.of("ORANGE")), - ImmutableList.of(functionCall("SUM", "APPLE")), + ImmutableList.of(functionCall("COUNT", "APPLE")), new TumblingWindowExpression(10, TimeUnit.SECONDS) ); @@ -158,6 +160,7 @@ public void shouldResolveSchemaForStreamWindowedAggregate() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .valueColumn(SchemaUtil.WINDOWSTART_NAME, SchemaUtil.WINDOWBOUND_TYPE) @@ -184,6 +187,7 @@ public void shouldResolveSchemaForStreamSelect() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("JUICE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("PLANTAIN"), SqlTypes.STRING) .valueColumn(ColumnName.of("CITRUS"), SqlTypes.INTEGER) @@ -207,6 +211,7 @@ public void shouldResolveSchemaForStreamFlatMap() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("APPLE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BANANA"), SqlTypes.STRING) @@ -343,6 +348,7 @@ public void shouldResolveSchemaForTableAggregate() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .build()) @@ -387,6 +393,7 @@ public void shouldResolveSchemaForTableSelect() { // Then: assertThat(result, is( LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("JUICE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("PLANTAIN"), SqlTypes.STRING) .valueColumn(ColumnName.of("CITRUS"), SqlTypes.INTEGER) @@ -460,7 +467,6 @@ private void givenAggregateFunction(final String name, final SqlType returnType) final KsqlAggregateFunction aggregateFunction = mock(KsqlAggregateFunction.class); when(functionRegistry.getAggregateFunction(eq(FunctionName.of(name)), any(), any())) .thenReturn(aggregateFunction); - when(aggregateFunction.name()).thenReturn(FunctionName.of(name)); when(aggregateFunction.getAggregateType()).thenReturn(SqlTypes.INTEGER); when(aggregateFunction.returnType()).thenReturn(returnType); when(aggregateFunction.getInitialValueSupplier()).thenReturn(mock(Supplier.class)); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java index 62f140497faa..abe495b14e38 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java @@ -30,7 +30,6 @@ import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.logging.processing.ProcessingLoggerFactory; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.name.SourceName; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; @@ -61,9 +60,8 @@ public class StreamGroupByBuilderTest { private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.STRING); - private static final SourceName ALIAS = SourceName.of("SOURCE"); - private static final LogicalSchema SCHEMA = LogicalSchema.builder() + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("PAC"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("MAN"), SqlTypes.STRING) .build() diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java index ada2ed4d50a9..dd06c335abca 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java @@ -63,7 +63,7 @@ public class StreamSelectKeyBuilderTest { private static final LogicalSchema SOURCE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("BIG"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BOI"), SqlTypes.BIGINT) .build() @@ -77,13 +77,11 @@ public class StreamSelectKeyBuilderTest { .valueColumn(ColumnName.of("BIG"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BOI"), SqlTypes.BIGINT) .valueColumn(ColumnName.of(SchemaUtil.ROWTIME_NAME.text()), SqlTypes.BIGINT) - // Note: Type of ROWKEY is old key's type: - .valueColumn(ColumnName.of(SchemaUtil.ROWKEY_NAME.text()), SqlTypes.STRING) + .valueColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .build(); private static final KeyBuilder RESULT_KEY_BUILDER = StructKeyUtil.keyBuilder(RESULT_SCHEMA); - private static final long A_BOI = 5000; private static final long A_BIG = 3000; diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java index ad6e30d2066f..53a4500e4d57 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java @@ -56,7 +56,9 @@ import org.mockito.junit.MockitoRule; public class TableGroupByBuilderTest { + private static final LogicalSchema SCHEMA = LogicalSchema.builder() + .keyColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("PAC"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("MAN"), SqlTypes.STRING) .build() @@ -67,9 +69,6 @@ public class TableGroupByBuilderTest { .valueColumns(SCHEMA.value()) .build(); - private static final PhysicalSchema PHYSICAL_SCHEMA = - PhysicalSchema.from(SCHEMA, SerdeOption.none()); - private static final PhysicalSchema REKEYED_PHYSICAL_SCHEMA = PhysicalSchema.from(REKEYED_SCHEMA, SerdeOption.none()); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java index 4136240bddec..6555e7b53051 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java @@ -41,7 +41,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.SchemaUtil; import java.time.Instant; import java.util.List; import java.util.Optional; @@ -60,7 +59,7 @@ public class KsqlMaterializationTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v1"), SqlTypes.STRING) .build();