Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Persistent queries on tables should require key columns #5303

Closed
big-andy-coates opened this issue May 7, 2020 · 1 comment · Fixed by #5533
Closed

Persistent queries on tables should require key columns #5303

big-andy-coates opened this issue May 7, 2020 · 1 comment · Fixed by #5533
Milestone

Comments

@big-andy-coates
Copy link
Contributor

As per discussion here:

#5115 (comment)

We should require transient queries on tables to include the primary key(s), so that any transient push query can be converted to a peristent query.

@big-andy-coates
Copy link
Contributor Author

big-andy-coates commented May 14, 2020

This will require a (breaking) change to the data sent back from the rest API. Likely something like:

{
      "name": "table tombstones",
      "statements": [
        "CREATE STREAM INPUT (K BIGINT KEY, V0 INT) WITH (kafka_topic='test_topic', value_format='JSON');",
        "CREATE TABLE T AS SELECT K, SUM(V0) AS SUM FROM INPUT GROUP BY K HAVING SUM(V0) > 0;",
        "SELECT * FROM INPUT EMIT CHANGES LIMIT 3;"
      ],
      "inputs": [
        {"topic": "test_topic", "key": 11, "value": {"v0": 1}},
        {"topic": "test_topic", "key": 11, "value": {"v0": -2}},
        {"topic": "test_topic", "key": 11, "value": {"v0": 10}}
      ],
      "responses": [
        {"admin": {"@type": "currentStatus"}},
        {"query": [
          {"header":{"schema":"``K` STRING KEY, `SUM` BIGINT"}},
          {"row":{"keys":[11], "values":[1]}},
          {"row":{"keys":[11], "tombstone":true}},
          {"row":{"keys":[11], "values":[9]}},
          {"finalMessage":"Limit Reached"}
        ]}
      ]
    }

Note the difference in the row output rows.

[
          {"row":{"keys":[11], "values":[1]}},
          {"row":{"keys":[11], "tombstone":true}}
]

vs current:

[
          {"row":{"values":[11, 1]}}
]

i.e. we'll need to split the key column value(s) out of the value column value(s). Then either explicitly set a tombstone flag, or just have this implicit from the lack of a values field.

@big-andy-coates big-andy-coates moved this from Backlog to In progress in Primitive Keys Jun 3, 2020
big-andy-coates added a commit to big-andy-coates/ksql that referenced this issue Jun 3, 2020
implements: [KLIP-29](confluentinc#5530)

fixes: confluentinc#5303
fixes: confluentinc#4678

This change sees ksqlDB no longer adding an implicit `ROWKEY STRING` key column to created streams or primary key column to created tables when no key column is explicitly provided in the `CREATE` statement.

BREAKING CHANGE

`CREATE TABLE` statements will now fail if not `PRIMARY KEY` column is provided.

For example, a statement such as:

```sql
CREATE TABLE FOO (name STRING) WITH (kafka_topic='foo', value_format='json');
```

Will need to be updated to include the definition of the PRIMARY KEY, e.g.

```sql
CREATE TABLE FOO (ID STRING PRIMARY KEY, name STRING) WITH (kafka_topic='foo', value_format='json');
```

If using schema inference, i.e. loading the value columns of the topic from the Schema Registry, the primary key can be provided as a partial schema, e.g.

```sql
-- FOO will have value columns loaded from the Schema Registry
CREATE TABLE FOO (ID INT PRIMARY KEY) WITH (kafka_topic='foo', value_format='avro');
```

`CREATE STREAM` statements that do not define a `KEY` column will no longer have an implicit `ROWKEY` key column.

For example:

```sql
CREATE STREAM BAR (NAME STRING) WITH (...);
```

The above statement would previously have resulted in a stream with two columns: `ROWKEY STRING KEY` and `NAME STRING`.
With this change the above statement will result in a stream with only the `NAME STRING` column.

Streams will no KEY column will be serialized to Kafka topics with a `null` key.
@big-andy-coates big-andy-coates mentioned this issue Jun 3, 2020
2 tasks
Primitive Keys automation moved this from In progress to Done Jun 3, 2020
big-andy-coates added a commit that referenced this issue Jun 3, 2020
* feat: explicit keys

implements: [KLIP-29](#5530)

fixes: #5303
fixes: #4678

This change sees ksqlDB no longer adding an implicit `ROWKEY STRING` key column to created streams or primary key column to created tables when no key column is explicitly provided in the `CREATE` statement.

BREAKING CHANGE

`CREATE TABLE` statements will now fail if not `PRIMARY KEY` column is provided.

For example, a statement such as:

```sql
CREATE TABLE FOO (name STRING) WITH (kafka_topic='foo', value_format='json');
```

Will need to be updated to include the definition of the PRIMARY KEY, e.g.

```sql
CREATE TABLE FOO (ID STRING PRIMARY KEY, name STRING) WITH (kafka_topic='foo', value_format='json');
```

If using schema inference, i.e. loading the value columns of the topic from the Schema Registry, the primary key can be provided as a partial schema, e.g.

```sql
-- FOO will have value columns loaded from the Schema Registry
CREATE TABLE FOO (ID INT PRIMARY KEY) WITH (kafka_topic='foo', value_format='avro');
```

`CREATE STREAM` statements that do not define a `KEY` column will no longer have an implicit `ROWKEY` key column.

For example:

```sql
CREATE STREAM BAR (NAME STRING) WITH (...);
```

The above statement would previously have resulted in a stream with two columns: `ROWKEY STRING KEY` and `NAME STRING`.
With this change the above statement will result in a stream with only the `NAME STRING` column.

Streams will no KEY column will be serialized to Kafka topics with a `null` key.


Co-authored-by: Andy Coates <big-andy-coates@users.noreply.github.com>
stevenpyzhang pushed a commit that referenced this issue Jun 5, 2020
* feat: explicit keys

implements: [KLIP-29](#5530)

fixes: #5303
fixes: #4678

This change sees ksqlDB no longer adding an implicit `ROWKEY STRING` key column to created streams or primary key column to created tables when no key column is explicitly provided in the `CREATE` statement.

BREAKING CHANGE

`CREATE TABLE` statements will now fail if not `PRIMARY KEY` column is provided.

For example, a statement such as:

```sql
CREATE TABLE FOO (name STRING) WITH (kafka_topic='foo', value_format='json');
```

Will need to be updated to include the definition of the PRIMARY KEY, e.g.

```sql
CREATE TABLE FOO (ID STRING PRIMARY KEY, name STRING) WITH (kafka_topic='foo', value_format='json');
```

If using schema inference, i.e. loading the value columns of the topic from the Schema Registry, the primary key can be provided as a partial schema, e.g.

```sql
-- FOO will have value columns loaded from the Schema Registry
CREATE TABLE FOO (ID INT PRIMARY KEY) WITH (kafka_topic='foo', value_format='avro');
```

`CREATE STREAM` statements that do not define a `KEY` column will no longer have an implicit `ROWKEY` key column.

For example:

```sql
CREATE STREAM BAR (NAME STRING) WITH (...);
```

The above statement would previously have resulted in a stream with two columns: `ROWKEY STRING KEY` and `NAME STRING`.
With this change the above statement will result in a stream with only the `NAME STRING` column.

Streams will no KEY column will be serialized to Kafka topics with a `null` key.

Co-authored-by: Andy Coates <big-andy-coates@users.noreply.github.com>
@big-andy-coates big-andy-coates modified the milestones: 0.11.0, 0.10.0 Jul 8, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
Development

Successfully merging a pull request may close this issue.

1 participant