diff --git a/design-proposals/klip-29-explicit-keys.md b/design-proposals/klip-29-explicit-keys.md index 852ff3f4777b..6c723d7e81b3 100644 --- a/design-proposals/klip-29-explicit-keys.md +++ b/design-proposals/klip-29-explicit-keys.md @@ -2,7 +2,7 @@ **Author**: @big-andy-coates | **Release Target**: 0.10.0 | -**Status**: In Discussion | +**Status**: Merged | **Discussion**: [Github PR](https://github.com/confluentinc/ksql/pull/5530) **tl;dr:** Up until now ksqlDB has added an implicit `ROWKEY STRING (PRIMARY) KEY` to a `CREATE TABLE` diff --git a/docs/developer-guide/create-a-stream.md b/docs/developer-guide/create-a-stream.md index af8245a0ae5b..674620183712 100644 --- a/docs/developer-guide/create-a-stream.md +++ b/docs/developer-guide/create-a-stream.md @@ -90,7 +90,6 @@ Your output should resemble: Name : PAGEVIEWS Field | Type -------------------------------------- - ROWKEY | VARCHAR(STRING) (key) VIEWTIME | BIGINT USERID | VARCHAR(STRING) PAGEID | VARCHAR(STRING) @@ -98,23 +97,15 @@ Name : PAGEVIEWS For runtime statistics and query details run: DESCRIBE EXTENDED ; ``` -You may notice that ksqlDB has added a key column named `ROWKEY`. -This is the default key column that ksqlDB adds if you don't provide one. -If your data doesn't contain a {{ site.ak }} serialized -`STRING` in the {{ site.ak }} message key, don't use `ROWKEY` in your SQL statements, -because this may cause unexpected results. - ### Create a Stream with a Specified Key The previous SQL statement doesn't define a column to represent the data in the -{{ site.ak }} message key in the underlying {{ site.ak }} topic, so the system adds a -`ROWKEY` column with type `STRING`. - -If the {{ site.ak }} message key is serialized in a key format that ksqlDB supports (currently `KAFKA`), +{{ site.ak }} message key in the underlying {{ site.ak }} topic. If the {{ site.ak }} message key +is serialized in a key format that ksqlDB supports (currently `KAFKA`), you can specify the key in the column list of the CREATE STREAM statement. -For example, the {{ site.ak }} message key of the `pageviews` topic is a `BIGINT` containing the `viewtime`, -so you can write the CREATE STREAM statement like this: +For example, the {{ site.ak }} message key of the `pageviews` topic is a `BIGINT` containing the +`viewtime`, so you can write the CREATE STREAM statement like this: ```sql CREATE STREAM pageviews_withkey diff --git a/docs/developer-guide/ksqldb-reference/create-stream.md b/docs/developer-guide/ksqldb-reference/create-stream.md index 70db05653b89..5865e396080b 100644 --- a/docs/developer-guide/ksqldb-reference/create-stream.md +++ b/docs/developer-guide/ksqldb-reference/create-stream.md @@ -45,8 +45,7 @@ Each column is defined by: If a column is not marked as a `KEY` column, ksqlDB loads it from the Kafka message's value. Unlike a table's `PRIMARY KEY`, a stream's keys can be NULL. -ksqlDB adds an implicit `ROWKEY` system column to every stream and table, which represents the -corresponding Kafka message key. An implicit `ROWTIME` pseudo column is also available on every +ksqlDB adds an implicit `ROWTIME` pseudo column is also available on every stream and table, which represents the corresponding Kafka message timestamp. The timestamp has milliseconds accuracy, and generally represents the _event time_ of a stream row and the _last modified time_ of a table row. @@ -60,7 +59,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. 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. | @@ -83,10 +81,9 @@ Example ```sql CREATE STREAM pageviews ( - rowkey BIGINT KEY, + page_id BIGINT KEY, viewtime BIGINT, - user_id VARCHAR, - page_id VARCHAR + user_id VARCHAR ) WITH (VALUE_FORMAT = 'JSON', KAFKA_TOPIC = 'my-pageviews-topic'); ``` diff --git a/docs/developer-guide/ksqldb-reference/create-table.md b/docs/developer-guide/ksqldb-reference/create-table.md index db7186f54c4e..29727a9085c3 100644 --- a/docs/developer-guide/ksqldb-reference/create-table.md +++ b/docs/developer-guide/ksqldb-reference/create-table.md @@ -44,9 +44,8 @@ Each column is defined by: `PRIMARY KEY` columns. If a column is not marked as a `PRIMARY KEY` column ksqlDB loads it from the Kafka message's value. Unlike a stream's `KEY` column, a table's `PRIMARY KEY` column(s) are NON NULL. Any records in the Kafka topic with NULL key columns are dropped. - -ksqlDB adds an implicit `ROWKEY` system column to every stream and table, which represents the -corresponding Kafka message key. An implicit `ROWTIME` pseudo column is also available on every + +ksqlDB adds an implicit `ROWTIME` pseudo column is also available on every stream and table, which represents the corresponding Kafka message timestamp. The timestamp has milliseconds accuracy, and generally represents the _event time_ of a stream row and the _last modified time_ of a table row. @@ -60,7 +59,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. 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. | @@ -79,9 +77,8 @@ Example ```sql CREATE TABLE users ( - rowkey BIGINT PRIMARY KEY, + id BIGINT PRIMARY KEY, usertimestamp BIGINT, - user_id VARCHAR, gender VARCHAR, region_id VARCHAR ) diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java index 0b0f2a76abda..2803866a79c0 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java @@ -117,6 +117,22 @@ public CreateTableCommand createTableCommand( final SourceName sourceName = statement.getName(); final KsqlTopic topic = buildTopic(statement.getProperties(), serviceContext); final LogicalSchema schema = buildSchema(statement.getElements()); + if (schema.key().isEmpty()) { + final boolean usingSchemaInference = statement.getProperties().getSchemaId().isPresent(); + + final String additional = usingSchemaInference + ? System.lineSeparator() + + "Use a partial schema to define the primary key and still load the value columns from " + + "the Schema Registry, for example:" + + System.lineSeparator() + + "\tCREATE TABLE " + statement.getName().text() + " (ID INT PRIMARY KEY) WITH (...);" + : ""; + + throw new KsqlException( + "Tables require a PRIMARY KEY. Please define the PRIMARY KEY." + additional + ); + } + final Optional timestampColumn = buildTimestampColumn( ksqlConfig, statement.getProperties(), @@ -153,7 +169,7 @@ private static LogicalSchema buildSchema(final TableElements tableElements) { } }); - return tableElements.toLogicalSchema(true); + return tableElements.toLogicalSchema(); } private static KsqlTopic buildTopic( diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/execution/json/PlanJsonMapper.java b/ksqldb-engine/src/main/java/io/confluent/ksql/execution/json/PlanJsonMapper.java index 858b1a476c40..ba48de02c254 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/execution/json/PlanJsonMapper.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/execution/json/PlanJsonMapper.java @@ -38,7 +38,7 @@ public enum PlanJsonMapper { new JavaTimeModule(), new KsqlParserSerializationModule(), new KsqlTypesSerializationModule(), - new KsqlTypesDeserializationModule(true) + new KsqlTypesDeserializationModule() ) .enable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) .enable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES) diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java index de5fc39c3fdf..ff28193f743f 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java @@ -128,6 +128,11 @@ public Stream resolveSelectStar( @Override void validateKeyPresent(final SourceName sinkName, final Projection projection) { + if (getSchema().key().isEmpty()) { + // No key column. + return; + } + final ColumnName keyName = Iterables.getOnlyElement(getSchema().key()).name(); if (!projection.containsExpression(new QualifiedColumnReferenceExp(getAlias(), keyName)) diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java index b335b9e5cc0b..508496cfed9d 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java @@ -68,7 +68,7 @@ private void registerForCreateSource(final ConfiguredStatement expressions) { // Note: A repartition is only not required if partitioning by the existing key column, or // the existing keyField. + if (schema.key().isEmpty()) { + // No current key, so repartition needed: + return false; + } + if (schema.key().size() != 1) { throw new UnsupportedOperationException("logic only supports single key column"); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java index 1b7206d1ecf7..775dff5c4382 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java @@ -77,6 +77,10 @@ public class CommandFactoriesTest { private static final TableElement ELEMENT1 = tableElement(Namespace.VALUE, "bob", new Type(SqlTypes.STRING)); private static final TableElements SOME_ELEMENTS = TableElements.of(ELEMENT1); + private static final TableElements ELEMENTS_WITH_PK = TableElements.of( + tableElement(Namespace.PRIMARY_KEY, "k", new Type(SqlTypes.STRING)), + ELEMENT1 + ); private static final String TOPIC_NAME = "some topic"; private static final Map MINIMIM_PROPS = ImmutableMap.of( CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("JSON"), @@ -325,7 +329,7 @@ public void shouldCreateTableCommandWithSingleValueWrappingFromOverridesNotConfi ); final DdlStatement statement = - new CreateTable(SOME_NAME, SOME_ELEMENTS, true, withProperties); + new CreateTable(SOME_NAME, ELEMENTS_WITH_PK, true, withProperties); // When: final DdlCommand cmd = commandFactories diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java index f2d78e1a8d5d..2791a8ac53d6 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java @@ -19,6 +19,7 @@ import static io.confluent.ksql.model.WindowType.SESSION; import static io.confluent.ksql.model.WindowType.TUMBLING; import static io.confluent.ksql.parser.tree.TableElement.Namespace.KEY; +import static io.confluent.ksql.parser.tree.TableElement.Namespace.PRIMARY_KEY; import static io.confluent.ksql.parser.tree.TableElement.Namespace.VALUE; import static io.confluent.ksql.schema.ksql.ColumnMatchers.keyColumn; import static io.confluent.ksql.schema.ksql.SystemColumns.ROWKEY_NAME; @@ -105,7 +106,10 @@ public class CreateSourceFactoryTest { private static final TableElement ELEMENT2 = tableElement(VALUE, "hojjat", new Type(BIGINT)); - private static final TableElements ONE_ELEMENTS = TableElements.of(ELEMENT1); + private static final TableElements ONE_ELEMENT = TableElements.of(ELEMENT1); + + private static final TableElements TABLE_ELEMENTS_1_VALUE = + TableElements.of(EXPLICIT_PRIMARY_KEY, ELEMENT1); private static final TableElements TABLE_ELEMENTS = TableElements.of(EXPLICIT_PRIMARY_KEY, ELEMENT1, ELEMENT2); @@ -192,7 +196,7 @@ public void shouldCreateCommandForCreateTable() { // Given: final CreateTable ddlStatement = new CreateTable(SOME_NAME, TableElements.of( - tableElement(VALUE, "COL1", new Type(BIGINT)), + tableElement(PRIMARY_KEY, "COL1", new Type(BIGINT)), tableElement(VALUE, "COL2", new Type(SqlTypes.STRING))), true, withProperties); @@ -219,7 +223,7 @@ public void shouldCreateStreamCommandWithSingleValueWrappingFromPropertiesNotCon givenProperty(CommonCreateConfigs.WRAP_SINGLE_VALUE, new BooleanLiteral("false")); final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory @@ -240,7 +244,7 @@ public void shouldCreateStreamCommandWithSingleValueWrappingFromConfig() { )); final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory @@ -254,7 +258,7 @@ public void shouldCreateStreamCommandWithSingleValueWrappingFromConfig() { public void shouldCreateStreamCommandWithSingleValueWrappingFromDefaultConfig() { // Given: final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory @@ -278,7 +282,7 @@ public void shouldCreateTableCommandWithSingleValueWrappingFromPropertiesNotConf givenProperty(CommonCreateConfigs.WRAP_SINGLE_VALUE, new BooleanLiteral("false")); final CreateTable statement = - new CreateTable(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateTable(SOME_NAME, TABLE_ELEMENTS_1_VALUE, true, withProperties); // When: final CreateTableCommand cmd = createSourceFactory @@ -298,7 +302,7 @@ public void shouldCreateTableCommandWithSingleValueWrappingFromConfig() { )); final CreateTable statement = - new CreateTable(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateTable(SOME_NAME, TABLE_ELEMENTS_1_VALUE, true, withProperties); // When: final CreateTableCommand cmd = createSourceFactory @@ -312,7 +316,7 @@ public void shouldCreateTableCommandWithSingleValueWrappingFromConfig() { public void shouldCreateTableCommandWithSingleValueWrappingFromDefaultConfig() { // Given: final CreateTable statement = - new CreateTable(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateTable(SOME_NAME, TABLE_ELEMENTS_1_VALUE, true, withProperties); // When: final CreateTableCommand cmd = createSourceFactory @@ -360,7 +364,7 @@ public void shouldThrowOnNoElementsInCreateTable() { public void shouldNotThrowWhenThereAreElementsInCreateStream() { // Given: final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: createSourceFactory.createStreamCommand(statement, ksqlConfig); @@ -372,7 +376,7 @@ public void shouldNotThrowWhenThereAreElementsInCreateStream() { public void shouldNotThrowWhenThereAreElementsInCreateTable() { // Given: final CreateTable statement = - new CreateTable(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateTable(SOME_NAME, TABLE_ELEMENTS_1_VALUE, true, withProperties); // When: createSourceFactory.createTableCommand(statement, ksqlConfig); @@ -385,7 +389,7 @@ public void shouldThrowIfTopicDoesNotExistForStream() { // Given: when(topicClient.isTopicExists(any())).thenReturn(false); final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final Exception e = assertThrows( @@ -403,7 +407,7 @@ public void shouldThrowIfTopicDoesNotExistForStream() { public void shouldNotThrowIfTopicDoesExist() { // Given: final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: createSourceFactory.createStreamCommand(statement, ksqlConfig); @@ -420,7 +424,7 @@ public void shouldThrowIfTimestampColumnDoesNotExistForStream() { new StringLiteral("`will-not-find-me`") ); final CreateStream statement = - new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final Exception e = assertThrows( @@ -438,9 +442,8 @@ public void shouldThrowIfTimestampColumnDoesNotExistForStream() { public void shouldBuildSerdeOptionsForStream() { // Given: givenCommandFactoriesWithMocks(); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) .valueColumn(ColumnName.of("bob"), SqlTypes.STRING) .build(); when(serdeOptionsSupplier.build(any(), any(), any(), any())).thenReturn(SOME_SERDE_OPTIONS); @@ -639,7 +642,7 @@ public void shouldCreateValueSerdeToValidateValueFormatCanHandleValueSchema() { @Test public void shouldDefaultToKafkaKeySerdeForStream() { - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory.createStreamCommand( @@ -658,7 +661,7 @@ public void shouldHandleValueAvroSchemaNameForStream() { givenCommandFactoriesWithMocks(); givenProperty("VALUE_FORMAT", new StringLiteral("Avro")); givenProperty("value_avro_schema_full_name", new StringLiteral("full.schema.name")); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory.createStreamCommand( @@ -676,7 +679,7 @@ public void shouldHandleValueAvroSchemaNameForStream() { public void shouldHandleSessionWindowedKeyForStream() { // Given: givenProperty("window_type", new StringLiteral("session")); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory.createStreamCommand( @@ -696,7 +699,7 @@ public void shouldHandleTumblingWindowedKeyForStream() { "window_type", new StringLiteral("tumbling"), "window_size", new StringLiteral("1 MINUTE") )); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory.createStreamCommand( @@ -719,7 +722,7 @@ public void shouldHandleHoppingWindowedKeyForStream() { "window_type", new StringLiteral("Hopping"), "window_size", new StringLiteral("2 SECONDS") )); - final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENTS, true, withProperties); + final CreateStream statement = new CreateStream(SOME_NAME, ONE_ELEMENT, true, withProperties); // When: final CreateStreamCommand cmd = createSourceFactory.createStreamCommand( @@ -875,6 +878,27 @@ public void shouldNotThrowOnKeyColumnThatIsNotCalledRowKey() { )); } + @Test + public void shouldThrowIfTableIsMissingPrimaryKey() { + // Given: + ksqlConfig = new KsqlConfig(ImmutableMap.of( + KsqlConfig.KSQL_WRAP_SINGLE_VALUES, false + )); + + final CreateTable statement = + new CreateTable(SOME_NAME, ONE_ELEMENT, true, withProperties); + + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> createSourceFactory.createTableCommand(statement, ksqlConfig) + ); + + // Then: + assertThat(e.getMessage(), + containsString("Tables require a PRIMARY KEY. Please define the PRIMARY KEY.")); + } + private void givenProperty(final String name, final Literal value) { givenProperties(ImmutableMap.of(name, value)); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/execution/json/PlanJsonMapperTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/execution/json/PlanJsonMapperTest.java index e3fa2d0637e6..16c8193b273d 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/execution/json/PlanJsonMapperTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/execution/json/PlanJsonMapperTest.java @@ -37,6 +37,6 @@ public void shouldEnableFailOnInvalidSubtype() { public void shouldHaveTypeMapperRegistered() { assertThat( MAPPER.getRegisteredModuleIds(), - hasItem(new KsqlTypesDeserializationModule(false).getTypeId())); + hasItem(new KsqlTypesDeserializationModule().getTypeId())); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java index 367db63dde07..5aa4705375cd 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/JsonFormatTest.java @@ -145,8 +145,10 @@ private void execInitCreateStreamQueries() { + ") WITH (value_format = 'json', " + "kafka_topic='" + inputTopic + "');"; - final String messageStreamStr = String.format("CREATE STREAM %s (message varchar) WITH (value_format = 'json', " - + "kafka_topic='%s');", messageLogStream, messageLogTopic); + final String messageStreamStr = String.format("CREATE STREAM %s " + + "(ROWKEY STRING KEY, message varchar) " + + "WITH (value_format = 'json', kafka_topic='%s');", + messageLogStream, messageLogTopic); KsqlEngineTestUtil.execute( serviceContext, ksqlEngine, ordersStreamStr, ksqlConfig, Collections.emptyMap()); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SecureIntegrationTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SecureIntegrationTest.java index e159cafcf556..c629fe0bf96b 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SecureIntegrationTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/SecureIntegrationTest.java @@ -353,7 +353,7 @@ private void awaitAsyncInputTopicCreation() { private void execInitCreateStreamQueries() { final String ordersStreamStr = - "CREATE STREAM " + INPUT_STREAM + " (ORDERTIME bigint, ORDERID varchar, " + "CREATE STREAM " + INPUT_STREAM + " (ROWKEY STRING KEY, ORDERTIME bigint, ORDERID varchar, " + "ITEMID varchar, ORDERUNITS double, PRICEARRAY array, KEYVALUEMAP " + "map) WITH (value_format = 'json', " + "kafka_topic='" + INPUT_TOPIC + "');"; diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java index 3d3b6b607e62..fd3aa482bd75 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java @@ -63,7 +63,7 @@ public class PhysicalPlanBuilderTest { private static final String CREATE_STREAM_TEST1 = "CREATE STREAM TEST1 " - + "(COL0 BIGINT, COL1 VARCHAR, COL2 DOUBLE) " + + "(ROWKEY STRING KEY, COL0 BIGINT, COL1 VARCHAR, COL2 DOUBLE) " + "WITH (KAFKA_TOPIC = 'test1', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST2 = "CREATE STREAM TEST2 " @@ -214,7 +214,7 @@ public void shouldCreateExecutionPlanForInsert() { @Test public void shouldCreatePlanForInsertIntoStreamFromStream() { // Given: - final String cs = "CREATE STREAM test1 (col0 INT) " + final String cs = "CREATE STREAM test1 (ROWKEY STRING KEY, col0 INT) " + "WITH (KAFKA_TOPIC='test1', VALUE_FORMAT='JSON');"; final String csas = "CREATE STREAM s0 AS SELECT * FROM test1;"; final String insertInto = "INSERT INTO s0 SELECT * FROM test1;"; diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java index 9bcc1311b43a..27662a39ce0a 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java @@ -84,7 +84,7 @@ private static void validate(final LogicalSchema schema) { throw new IllegalArgumentException("Schema contains system columns in value schema"); } - if (schema.key().size() != 1) { + if (schema.key().size() > 1) { throw new UnsupportedOperationException("Only single key columns supported. " + "Got: " + schema.key() + " (" + schema.key().size() + ")"); } diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java index c412a66be00f..7b13dbd86679 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateTableCommand.java @@ -45,6 +45,10 @@ public CreateTableCommand( formats, windowInfo ); + + if (schema.key().isEmpty()) { + throw new UnsupportedOperationException("Tables require key columns"); + } } @Override diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java index 9cae8b2b3a47..cf1f04349fdb 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/SourceNode.java @@ -130,7 +130,7 @@ private static Class toType(final String type) { private static LogicalSchema parseSchema(final String text) { return SchemaParser.parse(text, TypeRegistry.EMPTY) - .toLogicalSchema(true); + .toLogicalSchema(); } public static class Deserializer extends JsonDeserializer { diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/kafka/KafkaSerdeSupplier.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/kafka/KafkaSerdeSupplier.java index 6e985c566070..f5b2d2c2b801 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/kafka/KafkaSerdeSupplier.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/kafka/KafkaSerdeSupplier.java @@ -63,6 +63,19 @@ private SqlType getColumnType(final boolean isKey) { return columns.get(0).type(); } + private Serde getSerde(final boolean isKey) { + final List columns = isKey ? schema.key() : schema.value(); + if (columns.isEmpty()) { + return Serdes.Void(); + } + + if (columns.size() != 1) { + throw new IllegalStateException("KAFKA format only supports single column schemas."); + } + + return getSerde(columns.get(0).type()); + } + private static Serde getSerde(final SqlType sqlType) { final Type connectType = SchemaConverters.sqlToConnectConverter() .toConnectSchema(sqlType) @@ -89,8 +102,7 @@ private final class RowSerializer implements Serializer { @SuppressWarnings({"unchecked", "rawtypes"}) @Override public void configure(final Map configs, final boolean isKey) { - final SqlType sqlType = getColumnType(isKey); - delegate = (Serializer)getSerde(sqlType).serializer(); + delegate = (Serializer)getSerde(isKey).serializer(); delegate.configure(configs, isKey); } @@ -108,8 +120,7 @@ private final class RowDeserializer implements Deserializer { @Override public void configure(final Map configs, final boolean isKey) { this.type = isKey ? "key" : "value"; - final SqlType sqlType = getColumnType(isKey); - delegate = getSerde(sqlType).deserializer(); + delegate = getSerde(isKey).deserializer(); delegate.configure(configs, isKey); } diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestJsonMapper.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestJsonMapper.java index 3871d800b554..5c3280982948 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestJsonMapper.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestJsonMapper.java @@ -39,7 +39,7 @@ public enum TestJsonMapper { .registerModule(new JavaTimeModule()) .registerModule(new StructSerializationModule()) .registerModule(new KsqlTypesSerializationModule()) - .registerModule(new KsqlTypesDeserializationModule(true)) + .registerModule(new KsqlTypesDeserializationModule()) .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) .enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS) .enable(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN) diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TopicInfoCache.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TopicInfoCache.java index f8fca0b297d4..ed24afa1647d 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TopicInfoCache.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TopicInfoCache.java @@ -266,6 +266,11 @@ public Record coerceRecordKey( } private Function keyCoercer() { + if (schema.key().isEmpty()) { + // No key column: + return key -> null; + } + final SqlType keyType = schema .key() .get(0) diff --git a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/SchemaTranslationTest.java b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/SchemaTranslationTest.java index 77e747d2bded..86635f048938 100644 --- a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/SchemaTranslationTest.java +++ b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/SchemaTranslationTest.java @@ -45,7 +45,7 @@ public class SchemaTranslationTest { private static final String TOPIC_NAME = "TEST_INPUT"; private static final String OUTPUT_TOPIC_NAME = "TEST_OUTPUT"; private static final String DDL_STATEMENT = "CREATE STREAM " + TOPIC_NAME - + " WITH (KAFKA_TOPIC='" + TOPIC_NAME + "', VALUE_FORMAT='AVRO');"; + + " (ROWKEY STRING KEY) WITH (KAFKA_TOPIC='" + TOPIC_NAME + "', VALUE_FORMAT='AVRO');"; private static final Topic OUTPUT_TOPIC = new Topic(OUTPUT_TOPIC_NAME, Optional.empty()); diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/plan.json new file mode 100644 index 000000000000..a349a819e062 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 DOUBLE) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` DOUBLE", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` DOUBLE", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` DOUBLE" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/spec.json new file mode 100644 index 000000000000..9f753552be4d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/spec.json @@ -0,0 +1,77 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799866, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should coerce column types that are not directly supported", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4.0 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : 4.0 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "float" + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 DOUBLE" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_coerce_column_types_that_are_not_directly_supported/6.0.0_1591141799866/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/plan.json new file mode 100644 index 000000000000..911c1355be20 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` STRING" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/spec.json new file mode 100644 index 000000000000..b6648ce4983d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/spec.json @@ -0,0 +1,81 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799750, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should convert enum to STRING", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : "SPADES" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : "SPADES" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : { + "type" : "enum", + "name" : "Suit", + "symbols" : [ "SPADES", "HEARTS", "DIAMONDS", "CLUBS" ] + } + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_enum_to_STRING/6.0.0_1591141799750/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/plan.json new file mode 100644 index 000000000000..3b3976c1687f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 STRUCT) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` STRUCT<`F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` STRUCT<`F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` STRUCT<`F1` INTEGER>" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/spec.json new file mode 100644 index 000000000000..6a83f403179c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/spec.json @@ -0,0 +1,88 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799840, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "should convert record to STRUCT", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : { + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : { + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : { + "type" : "record", + "name" : "something", + "fields" : [ { + "name" : "f1", + "type" : "int" + } ] + } + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 STRUCT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_convert_record_to_STRUCT/6.0.0_1591141799840/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/plan.json new file mode 100644 index 000000000000..5a619e1fe9b0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (EXPECTED INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`EXPECTED` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`EXPECTED` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`EXPECTED` INTEGER" + }, + "selectExpressions" : [ "EXPECTED AS EXPECTED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/spec.json new file mode 100644 index 000000000000..9446508ebe35 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/spec.json @@ -0,0 +1,89 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799894, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should filter out columns with unsupported types", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "expected" : 1, + "c2" : null, + "c3" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "EXPECTED" : 1 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "expected", + "type" : "int" + }, { + "name" : "c1", + "type" : [ "null", "bytes" ] + }, { + "name" : "c2", + "type" : [ "null", { + "type" : "fixed", + "name" : "md5", + "size" : 16 + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "EXPECTED INT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141799894/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/plan.json new file mode 100644 index 000000000000..84a6a0eaac6c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 ARRAY) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` ARRAY" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/spec.json new file mode 100644 index 000000000000..025f3514808b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/spec.json @@ -0,0 +1,80 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799778, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "should support arrays", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : [ "a", "", "Bc" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : [ "a", "", "Bc" ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : { + "type" : "array", + "items" : "string" + } + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 ARRAY" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_arrays/6.0.0_1591141799778/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/plan.json new file mode 100644 index 000000000000..08f2077e7e1f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 MAP) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` MAP", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` MAP" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/spec.json new file mode 100644 index 000000000000..2f705062d328 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/spec.json @@ -0,0 +1,84 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799808, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "should support maps", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : { + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : { + "f1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : { + "type" : "map", + "values" : "long" + } + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_maps/6.0.0_1591141799808/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/plan.json new file mode 100644 index 000000000000..5e5298d4a718 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, C1 BOOLEAN, C2 INTEGER, C3 BIGINT, C4 DOUBLE, C5 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "C1 AS C1", "C2 AS C2", "C3 AS C3", "C4 AS C4", "C5 AS C5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/spec.json new file mode 100644 index 000000000000..f15edd17de87 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/spec.json @@ -0,0 +1,97 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799916, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should support partial schemas", + "inputs" : [ { + "topic" : "input", + "key" : 1, + "value" : { + "c1" : true, + "c2" : 1, + "c3" : 400000000000, + "c4" : 1.284765648, + "c5" : "hello" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "C1" : true, + "C2" : 1, + "C3" : 400000000000, + "C4" : 1.284765648, + "C5" : "hello" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "boolean" + }, { + "name" : "c2", + "type" : "int" + }, { + "name" : "c3", + "type" : "long" + }, { + "name" : "c4", + "type" : "double" + }, { + "name" : "c5", + "type" : "string" + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (ID INT KEY) WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "ID INT KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_partial_schemas/6.0.0_1591141799916/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/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/plan.json new file mode 100644 index 000000000000..7f4c74b52f05 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 BOOLEAN, C2 INTEGER, C3 BIGINT, C4 DOUBLE, C5 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING" + }, + "selectExpressions" : [ "C1 AS C1", "C2 AS C2", "C3 AS C3", "C4 AS C4", "C5 AS C5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/spec.json new file mode 100644 index 000000000000..96c7c7c4ce55 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/spec.json @@ -0,0 +1,97 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141799621, + "path" : "query-validation-tests/avro.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should support primitives", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : true, + "c2" : 1, + "c3" : 400000000000, + "c4" : 1.284765648, + "c5" : "hello" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : true, + "C2" : 1, + "C3" : 400000000000, + "C4" : 1.284765648, + "C5" : "hello" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "boolean" + }, { + "name" : "c2", + "type" : "int" + }, { + "name" : "c3", + "type" : "long" + }, { + "name" : "c4", + "type" : "double" + }, { + "name" : "c5", + "type" : "string" + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/avro_-_should_support_primitives/6.0.0_1591141799621/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/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/plan.json new file mode 100644 index 000000000000..dea89a4e893a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (IGNORED STRING) WITH (KAFKA_TOPIC='test', PARTITIONS=1, VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`IGNORED` STRING", + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT ARRAY[STRUCT(A:=123), STRUCT(A:=123)] KSQL_COL_0\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_0` ARRAY>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`IGNORED` STRING" + }, + "selectExpressions" : [ "ARRAY[STRUCT(A:=123), STRUCT(A:=123)] AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/spec.json new file mode 100644 index 000000000000..2d3db90d31de --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/spec.json @@ -0,0 +1,67 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141806443, + "path" : "query-validation-tests/create-struct.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT>> NOT NULL" + }, + "testCase" : { + "name" : "duplicate structs in array", + "inputs" : [ { + "topic" : "test", + "key" : "", + "value" : { + "col1" : "hello world" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "KSQL_COL_0" : [ { + "A" : 123 + }, { + "A" : 123 + } ] + } + } ], + "topics" : [ { + "name" : "test", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (ignored VARCHAR) WITH (kafka_topic='test',value_format='json',partitions=1);", "CREATE STREAM OUTPUT AS SELECT array[struct(a:=123),struct(a:=123)] from INPUT emit changes;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/topology new file mode 100644 index 000000000000..9396b997ae94 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/create-struct_-_duplicate_structs_in_array/6.0.0_1591141806443/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/plan.json new file mode 100644 index 000000000000..52a45d378b18 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER, F0 INTEGER) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER, `F0` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER, `F0` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER, `F0` INTEGER" + }, + "selectExpressions" : [ "ID AS ID", "F0 AS F0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/spec.json new file mode 100644 index 000000000000..c61f6325140d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/spec.json @@ -0,0 +1,88 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141810128, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "stream without key column", + "inputs" : [ { + "topic" : "input", + "key" : null, + "value" : { + "ID" : 1, + "F0" : 2 + } + },{ + "topic" : "input", + "key" : "foo", + "value" : { + "ID" : 2, + "F0" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "ID" : 1, + "F0" : 2 + } + },{ + "topic" : "OUTPUT", + "key" : null, + "value" : { + "ID" : 2, + "F0" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (ID INT, F0 INT) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "ID INT, F0 INT" + }, { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "ID INT, F0 INT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_stream_without_key_column/6.0.0_1591141810128/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/plan.json new file mode 100644 index 000000000000..fb0f3e46f531 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 ARRAY) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`V0` ARRAY" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/spec.json new file mode 100644 index 000000000000..9da466878c35 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/spec.json @@ -0,0 +1,78 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809077, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate array element OK - AVRO", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "V0", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 ARRAY) WITH (kafka_topic='input', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_AVRO/6.0.0_1591141809077/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/plan.json new file mode 100644 index 000000000000..09591c926370 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 ARRAY) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`V0` ARRAY" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/spec.json new file mode 100644 index 000000000000..dfab0f341d4d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/spec.json @@ -0,0 +1,63 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809028, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate array element OK - JSON", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 ARRAY) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON/6.0.0_1591141809028/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/plan.json new file mode 100644 index 000000000000..ef237720707c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 ARRAY) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "sourceSchema" : "`V0` ARRAY" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/spec.json new file mode 100644 index 000000000000..87a2eefcc947 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/spec.json @@ -0,0 +1,85 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809050, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate array element OK - JSON_SR", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "object", + "properties" : { + "V0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "array", + "items" : { + "oneOf" : [ { + "type" : "null" + }, { + "type" : "integer", + "connect.type" : "int32" + } ] + } + } ] + } + } + }, + "format" : "JSON", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 ARRAY) WITH (kafka_topic='input', value_format='JSON_SR');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_JSON_SR/6.0.0_1591141809050/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/plan.json new file mode 100644 index 000000000000..c5c54ba1312f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 ARRAY) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`V0` ARRAY" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/spec.json new file mode 100644 index 000000000000..7a2479196bcb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/spec.json @@ -0,0 +1,65 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809096, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate array element OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : [ 1 ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated int32 V0 = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 ARRAY) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_array_element_OK_-_PROTOBUF/6.0.0_1591141809096/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/plan.json new file mode 100644 index 000000000000..5558bb613dc2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 DECIMAL(15, 14)) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` DECIMAL(15, 14)", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` DECIMAL(15, 14)", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`V0` DECIMAL(15, 14)" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/spec.json new file mode 100644 index 000000000000..53fe22df091b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/spec.json @@ -0,0 +1,63 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808862, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "validate decimal elements OK - JSON", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : 1.12345678901234 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : 1.12345678901234 + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 DECIMAL(15,14)) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON/6.0.0_1591141808862/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/plan.json new file mode 100644 index 000000000000..645ac2b6e260 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 DECIMAL(15, 14)) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` DECIMAL(15, 14)", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` DECIMAL(15, 14)", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "sourceSchema" : "`V0` DECIMAL(15, 14)" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/spec.json new file mode 100644 index 000000000000..000fef04a657 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/spec.json @@ -0,0 +1,84 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808879, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "validate decimal elements OK - JSON_SR", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : 1.12345678901234 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : 1.12345678901234 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "object", + "properties" : { + "V0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "number", + "title" : "org.apache.kafka.connect.data.Decimal", + "connect.version" : 1, + "connect.type" : "bytes", + "connect.parameters" : { + "scale" : "14", + "connect.decimal.precision" : "15" + } + } ] + } + } + }, + "format" : "JSON", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 DECIMAL(15,14)) WITH (kafka_topic='input', value_format='JSON_SR');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_decimal_elements_OK_-_JSON_SR/6.0.0_1591141808879/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/plan.json new file mode 100644 index 000000000000..daa320b8b103 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 MAP) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` MAP", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`V0` MAP" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/spec.json new file mode 100644 index 000000000000..75f16aa88033 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/spec.json @@ -0,0 +1,96 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809133, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate map element OK - AVRO", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "V0", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_V0", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_V0" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 MAP) WITH (kafka_topic='input', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_AVRO/6.0.0_1591141809133/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/plan.json new file mode 100644 index 000000000000..1f132250db7b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 MAP) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` MAP", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`V0` MAP" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/spec.json new file mode 100644 index 000000000000..23cb9dca802c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/spec.json @@ -0,0 +1,67 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809114, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate map element OK - JSON", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 MAP) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_JSON/6.0.0_1591141809114/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/plan.json new file mode 100644 index 000000000000..85b400aa301d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 MAP) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` MAP", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`V0` MAP" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/spec.json new file mode 100644 index 000000000000..46213373a476 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/spec.json @@ -0,0 +1,69 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809156, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate map element OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "k1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated ConnectDefault2Entry V0 = 1;\n\n message ConnectDefault2Entry {\n string key = 1;\n int32 value = 2;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 MAP) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_map_element_OK_-_PROTOBUF/6.0.0_1591141809156/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/plan.json new file mode 100644 index 000000000000..982d9c2d567e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 STRUCT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/spec.json new file mode 100644 index 000000000000..8feb09221fce --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/spec.json @@ -0,0 +1,94 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809223, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate struct element OK - AVRO", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "f0" : "bob", + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "F0" : "bob", + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "V0", + "type" : [ "null", { + "type" : "record", + "name" : "KsqlDataSourceSchema_V0", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "F1", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_V0" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 STRUCT) WITH (kafka_topic='input', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_AVRO/6.0.0_1591141809223/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/plan.json new file mode 100644 index 000000000000..5554a5edee0c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 STRUCT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/spec.json new file mode 100644 index 000000000000..e0692c38c962 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/spec.json @@ -0,0 +1,69 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809176, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate struct element OK - JSON", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "f0" : "bob", + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "F0" : "bob", + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 STRUCT) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON/6.0.0_1591141809176/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/plan.json new file mode 100644 index 000000000000..5b49e399f5e4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 STRUCT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "sourceSchema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/spec.json new file mode 100644 index 000000000000..81f366b03c38 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/spec.json @@ -0,0 +1,102 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809196, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate struct element OK - JSON_SR", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "f0" : "bob", + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "F0" : "bob", + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "object", + "properties" : { + "V0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "object", + "properties" : { + "F0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "string" + } ] + }, + "F1" : { + "connect.index" : 1, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "integer", + "connect.type" : "int32" + } ] + } + } + } ] + } + } + }, + "format" : "JSON", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 STRUCT) WITH (kafka_topic='input', value_format='JSON_SR');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_JSON_SR/6.0.0_1591141809196/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/plan.json new file mode 100644 index 000000000000..ff87463b84f5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (V0 STRUCT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`V0` STRUCT<`F0` STRING, `F1` INTEGER>" + }, + "selectExpressions" : [ "V0 AS V0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/spec.json new file mode 100644 index 000000000000..936d8d640dbf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/spec.json @@ -0,0 +1,71 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141809244, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "validate struct element OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "V0" : { + "f0" : "bob", + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "V0" : { + "F0" : "bob", + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n ConnectDefault2 V0 = 1;\n\n message ConnectDefault2 {\n string F0 = 1;\n int32 F1 = 2;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (V0 STRUCT) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_struct_element_OK_-_PROTOBUF/6.0.0_1591141809244/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/plan.json new file mode 100644 index 000000000000..db095fdc6930 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AvRo');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` INTEGER" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/spec.json new file mode 100644 index 000000000000..f781627e8d9c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808021, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "validate without elements OK - AVRO", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "int" + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='AvRo');", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_AVRO/6.0.0_1591141808021/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/plan.json new file mode 100644 index 000000000000..6a37ec3da86a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 BIGINT) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` BIGINT", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "sourceSchema" : "`C1` BIGINT" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/spec.json new file mode 100644 index 000000000000..c1c451203798 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808076, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "validate without elements OK - JSON_SR SCHEMA", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "object", + "properties" : { + "c1" : { + "type" : "integer" + } + } + }, + "format" : "JSON", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='JSON_SR');", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_JSON_SR_SCHEMA/6.0.0_1591141808076/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/plan.json new file mode 100644 index 000000000000..8efeb18c8fb7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` INTEGER" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/spec.json new file mode 100644 index 000000000000..771576fd9825 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/spec.json @@ -0,0 +1,65 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808046, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "validate without elements OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 c1 = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_OK_-_PROTOBUF/6.0.0_1591141808046/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/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/plan.json new file mode 100644 index 000000000000..44414788e86c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (`@TIMESTAMP` BIGINT, `FROM` BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`@TIMESTAMP` BIGINT, `FROM` BIGINT", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`@TIMESTAMP` BIGINT, `FROM` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`@TIMESTAMP` BIGINT, `FROM` BIGINT" + }, + "selectExpressions" : [ "`@TIMESTAMP` AS `@TIMESTAMP`", "`FROM` AS `FROM`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/spec.json new file mode 100644 index 000000000000..ebb410d5fc7a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/spec.json @@ -0,0 +1,70 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141808170, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<@TIMESTAMP BIGINT, FROM BIGINT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<@TIMESTAMP BIGINT, FROM BIGINT> NOT NULL" + }, + "testCase" : { + "name" : "with invalid or reserved words in schema - JSON", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "@timestamp" : 4, + "from" : 5 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "@TIMESTAMP" : 4, + "FROM" : 5 + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (`@TIMESTAMP` BIGINT, `FROM` BIGINT) WITH (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "`@TIMESTAMP` BIGINT, `FROM` BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_with_invalid_or_reserved_words_in_schema_-_JSON/6.0.0_1591141808170/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/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/plan.json new file mode 100644 index 000000000000..641ead26e62a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/plan.json @@ -0,0 +1,164 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRUCT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`DATA` STRUCT<`FIELD` INTEGER>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA->FIELD FIELD,\n COUNT(*) COUNT\nFROM TEST TEST\nGROUP BY TEST.DATA->FIELD\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`FIELD` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`DATA` STRUCT<`FIELD` INTEGER>" + }, + "selectExpressions" : [ "DATA AS DATA", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "DATA->FIELD" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "DATA", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "FIELD" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/spec.json new file mode 100644 index 000000000000..dc259d280860 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/spec.json @@ -0,0 +1,127 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141815037, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT, ROWTIME BIGINT> NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT, ROWTIME BIGINT, KSQL_AGG_VARIABLE_0 BIGINT> NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "int json field (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : 2 + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "COUNT" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (data STRUCT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT data->field, COUNT(*) AS COUNT FROM TEST GROUP BY data->field;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_json_field_(stream-_table)/6.0.0_1591141815037/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/plan.json new file mode 100644 index 000000000000..13b96c4b8081 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/plan.json @@ -0,0 +1,164 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRUCT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`DATA` STRUCT<`FIELD` STRING>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA->FIELD FIELD,\n COUNT(*) COUNT\nFROM TEST TEST\nGROUP BY TEST.DATA->FIELD\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`FIELD` STRING KEY, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`DATA` STRUCT<`FIELD` STRING>" + }, + "selectExpressions" : [ "DATA AS DATA", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "DATA->FIELD" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "DATA", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "FIELD" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/spec.json new file mode 100644 index 000000000000..4e29b1c8729d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/spec.json @@ -0,0 +1,127 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141814983, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT, ROWTIME BIGINT> NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT, ROWTIME BIGINT, KSQL_AGG_VARIABLE_0 BIGINT> NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "json field (stream->table)", + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : "Something" + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : "Something Else" + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { + "field" : "Something" + } + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : { } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "Something", + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Something Else", + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Something", + "value" : { + "COUNT" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (data STRUCT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT data->field AS FIELD, COUNT(*) AS COUNT FROM TEST GROUP BY data->field;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_json_field_(stream-_table)/6.0.0_1591141814983/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/plan.json new file mode 100644 index 000000000000..8c8f8d7b5c54 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/plan.json @@ -0,0 +1,164 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`DATA` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) COUNT\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`DATA` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`DATA` INTEGER" + }, + "selectExpressions" : [ "DATA AS DATA", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "DATA" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "DATA", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "DATA" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/spec.json new file mode 100644 index 000000000000..adc2f0ad0135 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141811525, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "steam with no key", + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : 22 + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : 333 + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "data" : 22 + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 22, + "value" : { + "ROWTIME" : 0, + "DATA" : 22 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 333, + "value" : { + "ROWTIME" : 0, + "DATA" : 333 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 22, + "value" : { + "ROWTIME" : 0, + "DATA" : 22 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 22, + "value" : { + "ROWTIME" : 0, + "DATA" : 22, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 333, + "value" : { + "ROWTIME" : 0, + "DATA" : 333, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 22, + "value" : { + "ROWTIME" : 0, + "DATA" : 22, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : 22, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 333, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 22, + "value" : { + "COUNT" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (data INT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE OUTPUT AS SELECT DATA, COUNT(*) AS COUNT FROM TEST GROUP BY DATA;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "table", + "schema" : "DATA INT KEY, COUNT BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_steam_with_no_key/6.0.0_1591141811525/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/plan.json new file mode 100644 index 000000000000..904451264875 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/plan.json @@ -0,0 +1,188 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE1 (K STRING, DATA STRING) WITH (KAFKA_TOPIC='stream-source', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE1", + "schema" : "`K` STRING, `DATA` STRING", + "topicName" : "stream-source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE2 (K STRING, DATA STRING) WITH (KAFKA_TOPIC='insert-source', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE2", + "schema" : "`K` STRING, `DATA` STRING", + "topicName" : "insert-source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM SOURCE1 SOURCE1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING, `DATA` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "SOURCE1" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "stream-source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING, `DATA` STRING" + }, + "selectExpressions" : [ "K AS K", "DATA AS DATA" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "INSERT INTO OUTPUT SELECT *\nFROM SOURCE2 SOURCE2\nEMIT CHANGES", + "queryPlan" : { + "sources" : [ "SOURCE2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "insert-source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING, `DATA` STRING" + }, + "selectExpressions" : [ "K AS K", "DATA AS DATA" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "INSERTQUERY_1" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/spec.json new file mode 100644 index 000000000000..053291f795f1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/spec.json @@ -0,0 +1,94 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141820194, + "path" : "query-validation-tests/insert-into.json", + "schemas" : { + "INSERTQUERY_1.KsqlTopic.Source" : "STRUCT NOT NULL", + "INSERTQUERY_1.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "streams with no key columns", + "inputs" : [ { + "topic" : "stream-source", + "key" : "", + "value" : { + "K" : "k1", + "data" : "v1" + } + }, { + "topic" : "insert-source", + "key" : "", + "value" : { + "K" : "k2", + "data" : "v2" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "K" : "k1", + "DATA" : "v1" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "K" : "k2", + "DATA" : "v2" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "stream-source", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "insert-source", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM SOURCE1 (K STRING, data VARCHAR) WITH (kafka_topic='stream-source', value_format='JSON');", "CREATE STREAM SOURCE2 (K STRING, data VARCHAR) WITH (kafka_topic='insert-source', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM SOURCE1;", "INSERT INTO OUTPUT SELECT * FROM SOURCE2;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "insert-source", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "stream-source", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/topology new file mode 100644 index 000000000000..d3d5f827f445 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_streams_with_no_key_columns/6.0.0_1591141820194/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [insert-source]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/plan.json new file mode 100644 index 000000000000..4f71d09edd0f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/plan.json @@ -0,0 +1,215 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (A INTEGER, B INTEGER, C INTEGER) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (A INTEGER, B INTEGER, C INTEGER) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM L L\nINNER JOIN R R WITHIN 10 SECONDS ON ((L.A = R.A))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L_A` INTEGER KEY, `L_B` INTEGER, `L_C` INTEGER, `R_A` INTEGER, `R_B` INTEGER, `R_C` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER" + }, + "keyExpression" : "A" + }, + "keyColumnNames" : [ "L_A" ], + "selectExpressions" : [ "A AS L_A", "B AS L_B", "C AS L_C", "ROWTIME AS L_ROWTIME" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER" + }, + "keyExpression" : "A" + }, + "keyColumnNames" : [ "R_A" ], + "selectExpressions" : [ "A AS R_A", "B AS R_B", "C AS R_C", "ROWTIME AS R_ROWTIME" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "keyColName" : "L_A" + }, + "keyColumnNames" : [ "L_A" ], + "selectExpressions" : [ "L_B AS L_B", "L_C AS L_C", "R_A AS R_A", "R_B AS R_B", "R_C AS R_C" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/spec.json new file mode 100644 index 000000000000..9d4a7f20de7f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/spec.json @@ -0,0 +1,143 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591176895543, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "streams with no key columns (stream->stream)", + "inputs" : [ { + "topic" : "LEFT", + "key" : "", + "value" : { + "A" : 0, + "B" : 1, + "C" : 2 + }, + "timestamp" : 10 + }, { + "topic" : "RIGHT", + "key" : "", + "value" : { + "A" : 0, + "B" : -1, + "C" : -2 + }, + "timestamp" : 11 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "R_A" : 0, + "L_B" : 1, + "R_B" : -1, + "L_C" : 2, + "R_C" : -2 + }, + "timestamp" : 11 + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (A INT, B INT, C INT) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE STREAM R (A INT, B INT, C INT) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM L INNER JOIN R WITHIN 10 SECONDS ON L.A = R.A;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "L_A INT KEY, L_B INT, L_C INT, R_A INT, R_B INT, R_C INT" + } ], + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/topology new file mode 100644 index 000000000000..0eee5fb318ac --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_stream)/6.0.0_1591176895543/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [RIGHT]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/plan.json new file mode 100644 index 000000000000..69b881ea723e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/plan.json @@ -0,0 +1,198 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (A INTEGER, B INTEGER, C INTEGER) WITH (KAFKA_TOPIC='LEFT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER", + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE R (A INTEGER PRIMARY KEY, B INTEGER, C INTEGER) WITH (KAFKA_TOPIC='RIGHT', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "R", + "schema" : "`A` INTEGER KEY, `B` INTEGER, `C` INTEGER", + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM L L\nINNER JOIN R R ON ((L.A = R.A))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L_A` INTEGER KEY, `L_B` INTEGER, `L_C` INTEGER, `R_A` INTEGER, `R_B` INTEGER, `R_C` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "LEFT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`A` INTEGER, `B` INTEGER, `C` INTEGER" + }, + "keyExpression" : "A" + }, + "keyColumnNames" : [ "L_A" ], + "selectExpressions" : [ "A AS L_A", "B AS L_B", "C AS L_C", "ROWTIME AS L_ROWTIME" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "RIGHT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`A` INTEGER KEY, `B` INTEGER, `C` INTEGER" + }, + "keyColumnNames" : [ "R_A" ], + "selectExpressions" : [ "B AS R_B", "C AS R_C", "ROWTIME AS R_ROWTIME", "A AS R_A" ] + }, + "keyColName" : "L_A" + }, + "keyColumnNames" : [ "L_A" ], + "selectExpressions" : [ "L_B AS L_B", "L_C AS L_C", "R_A AS R_A", "R_B AS R_B", "R_C AS R_C" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/spec.json new file mode 100644 index 000000000000..474811a2435b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/spec.json @@ -0,0 +1,111 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591176895633, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "streams with no key columns (stream->table)", + "inputs" : [ { + "topic" : "RIGHT", + "key" : 0, + "value" : { + "B" : -1, + "C" : -2 + }, + "timestamp" : 10 + }, { + "topic" : "LEFT", + "key" : "ignored", + "value" : { + "A" : 0, + "B" : 1, + "C" : 2 + }, + "timestamp" : 11 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "R_A" : 0, + "L_B" : 1, + "R_B" : -1, + "L_C" : 2, + "R_C" : -2 + }, + "timestamp" : 11 + } ], + "topics" : [ { + "name" : "LEFT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "RIGHT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (A INT, B INT, C INT) WITH (kafka_topic='LEFT', value_format='JSON');", "CREATE TABLE R (A INT PRIMARY KEY, B INT, C INT) WITH (kafka_topic='RIGHT', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM L INNER JOIN R ON L.A = R.A;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "L_A INT KEY, L_B INT, L_C INT, R_A INT, R_B INT, R_C INT" + } ], + "topics" : { + "topics" : [ { + "name" : "LEFT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "RIGHT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/topology new file mode 100644 index 000000000000..8f6443e49a08 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_streams_with_no_key_columns_(stream-_table)/6.0.0_1591176895633/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: Join-repartition-source (topics: [Join-repartition]) + --> Join + Source: KSTREAM-SOURCE-0000000000 (topics: [RIGHT]) + --> KTABLE-SOURCE-0000000001 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- Join-repartition-source + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Join + Sink: KSTREAM-SINK-0000000013 (topic: OUTPUT) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [LEFT]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasLeft (stores: []) + --> Join-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-repartition-filter (stores: []) + --> Join-repartition-sink + <-- PrependAliasLeft + Sink: Join-repartition-sink (topic: Join-repartition) + <-- Join-repartition-filter + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/plan.json new file mode 100644 index 000000000000..7217f6695ddf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (M1 MAP, M2 MAP) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`M1` MAP, `M2` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SINK AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SINK", + "schema" : "`M1` MAP, `M2` MAP", + "topicName" : "SINK", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "SINK", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SINK" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`M1` MAP, `M2` MAP" + }, + "selectExpressions" : [ "M1 AS M1", "M2 AS M2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "SINK" + }, + "queryId" : "CSAS_SINK_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/spec.json new file mode 100644 index 000000000000..405cee65af57 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/spec.json @@ -0,0 +1,123 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141839946, + "path" : "query-validation-tests/multiple-avro-maps.json", + "schemas" : { + "CSAS_SINK_0.KsqlTopic.Source" : "STRUCT, M2 MAP> NOT NULL", + "CSAS_SINK_0.SINK" : "STRUCT, M2 MAP> NOT NULL" + }, + "testCase" : { + "name" : "project multiple avro maps", + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : { + "M1" : { + "K1" : 123 + }, + "M2" : { + "K2" : "FOO" + } + } + } ], + "outputs" : [ { + "topic" : "SINK", + "key" : "0", + "value" : { + "M1" : { + "K1" : 123 + }, + "M2" : { + "K2" : "FOO" + } + } + } ], + "topics" : [ { + "name" : "SINK", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "M1", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_M1", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_M1" + } ], + "default" : null + }, { + "name" : "M2", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_M2", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_M2" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (M1 MAP, M2 MAP) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM SINK as SELECT * FROM test;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "SINK", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/topology new file mode 100644 index 000000000000..e799661da679 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/multiple-avro-maps_-_project_multiple_avro_maps/6.0.0_1591141839946/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> 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: SINK) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/plan.json new file mode 100644 index 000000000000..6ac332af6a0f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/plan.json @@ -0,0 +1,133 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (NAME STRING, ID INTEGER) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`NAME` STRING, `ID` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n INPUT.NAME NAME\nFROM INPUT INPUT\nPARTITION BY INPUT.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`NAME` STRING, `ID` INTEGER" + }, + "keyExpression" : "ID" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/spec.json new file mode 100644 index 000000000000..3e30fe72756a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/spec.json @@ -0,0 +1,69 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141840101, + "path" : "query-validation-tests/partition-by.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "no key column", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "NAME" : "bob", + "ID" : 10 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "bob" + } + } ], + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (NAME STRING, ID INT) with (kafka_topic='input', value_format='JSON');", "CREATE STREAM OUTPUT AS select ID, NAME from INPUT partition by ID;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "ID INT KEY, NAME STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/topology new file mode 100644 index 000000000000..a8325d21f26b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/partition-by_-_no_key_column/6.0.0_1591141840101/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PartitionBy-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/plan.json new file mode 100644 index 000000000000..03ebc531de17 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ASTR ARRAY, MSTR MAP) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ASTR` ARRAY, `MSTR` MAP", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ASTR` ARRAY, `MSTR` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`ASTR` ARRAY, `MSTR` MAP" + }, + "selectExpressions" : [ "ASTR AS ASTR", "MSTR AS MSTR" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/spec.json new file mode 100644 index 000000000000..95a5b96f5d14 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/spec.json @@ -0,0 +1,71 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141841906, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT, MSTR MAP> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT, MSTR MAP> NOT NULL" + }, + "testCase" : { + "name" : "protobuf containers", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "astr" : [ "1", "2" ], + "mstr" : { + "1" : "a" + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "ASTR" : [ "1", "2" ], + "MSTR" : { + "1" : "a" + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated string ASTR = 1;\n repeated ConnectDefault2Entry MSTR = 2;\n\n message ConnectDefault2Entry {\n string key = 1;\n string value = 2;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (astr ARRAY, mstr MAP) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_containers/6.0.0_1591141841906/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/plan.json new file mode 100644 index 000000000000..61538ce39731 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (S STRUCT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`S` STRUCT<`FOO` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`S` STRUCT<`FOO` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`S` STRUCT<`FOO` INTEGER>" + }, + "selectExpressions" : [ "S AS S" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/spec.json new file mode 100644 index 000000000000..36b1342e87c6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/spec.json @@ -0,0 +1,95 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141841955, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "protobuf defaults - empty struct fills defaults and nulls remain nulls", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "s" : { + "foo" : 0 + } + } + }, { + "topic" : "input", + "key" : "", + "value" : { + "s" : { } + } + }, { + "topic" : "input", + "key" : "", + "value" : { + "s" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "S" : { + "FOO" : 0 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "S" : { + "FOO" : 0 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "S" : null + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n ConnectDefault2 S = 1;\n\n message ConnectDefault2 {\n int32 FOO = 1;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (s STRUCT) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_empty_struct_fills_defaults_and_nulls_remain_nulls/6.0.0_1591141841955/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/plan.json new file mode 100644 index 000000000000..570c8fd82304 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (I ARRAY) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`I` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`I` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`I` ARRAY" + }, + "selectExpressions" : [ "I AS I" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/spec.json new file mode 100644 index 000000000000..6122eef4a960 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/spec.json @@ -0,0 +1,77 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141841939, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "protobuf defaults - nested nulls are defaulted to 0", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "i" : [ 0 ] + } + }, { + "topic" : "input", + "key" : "", + "value" : { + "i" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "I" : [ 0 ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "I" : [ ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated int32 I = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (i ARRAY) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT as SELECT * FROM input;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_defaults_-_nested_nulls_are_defaulted_to_0/6.0.0_1591141841939/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/plan.json new file mode 100644 index 000000000000..5d72ff2f736c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 DOUBLE, C2 BIGINT, C3 BIGINT, C4 INTEGER, C5 BIGINT) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` DOUBLE, `C2` BIGINT, `C3` BIGINT, `C4` INTEGER, `C5` BIGINT", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` DOUBLE, `C2` BIGINT, `C3` BIGINT, `C4` INTEGER, `C5` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` DOUBLE, `C2` BIGINT, `C3` BIGINT, `C4` INTEGER, `C5` BIGINT" + }, + "selectExpressions" : [ "C1 AS C1", "C2 AS C2", "C3 AS C3", "C4 AS C4", "C5 AS C5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/spec.json new file mode 100644 index 000000000000..7477d7c84f20 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/spec.json @@ -0,0 +1,78 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842093, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "protobuf inference - other scalars", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 1.1234, + "c2" : 1, + "c3" : 400000000000, + "c4" : 1, + "c5" : 400000000000 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : 1.1233999729156494, + "C2" : 1, + "C3" : 400000000000, + "C4" : 1, + "C5" : 400000000000 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n float c1 = 1;\n uint32 c2 = 2;\n uint64 c3 = 3;\n sint32 c4 = 4;\n sint64 c5 = 5;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 DOUBLE, C2 BIGINT, C3 BIGINT, C4 INT, C5 BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_other_scalars/6.0.0_1591141842093/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/plan.json new file mode 100644 index 000000000000..7b85308b6767 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (K STRING KEY, C1 BOOLEAN, C2 INTEGER, C3 BIGINT, C4 DOUBLE, C5 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`K` STRING KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`K` STRING KEY, `C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING" + }, + "keyColumnNames" : [ "K" ], + "selectExpressions" : [ "C1 AS C1", "C2 AS C2", "C3 AS C3", "C4 AS C4", "C5 AS C5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/spec.json new file mode 100644 index 000000000000..ce09bcbad148 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/spec.json @@ -0,0 +1,78 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141841999, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "protobuf inference - partital schema", + "inputs" : [ { + "topic" : "input", + "key" : "a", + "value" : { + "c1" : true, + "c2" : 1, + "c3" : 400000000000, + "c4" : 1.284765648, + "c5" : "hello" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : { + "C1" : true, + "C2" : 1, + "C3" : 400000000000, + "C4" : 1.284765648, + "C5" : "hello" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n bool c1 = 1;\n int32 c2 = 2;\n int64 c3 = 3;\n double c4 = 4;\n string c5 = 5;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (K STRING KEY) WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "K STRING KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_partital_schema/6.0.0_1591141841999/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/plan.json new file mode 100644 index 000000000000..1fb7113df8f3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 BOOLEAN, C2 INTEGER, C3 BIGINT, C4 DOUBLE, C5 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` BOOLEAN, `C2` INTEGER, `C3` BIGINT, `C4` DOUBLE, `C5` STRING" + }, + "selectExpressions" : [ "C1 AS C1", "C2 AS C2", "C3 AS C3", "C4 AS C4", "C5 AS C5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/spec.json new file mode 100644 index 000000000000..a15787e99b2f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/spec.json @@ -0,0 +1,78 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141841973, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "protobuf inference - primitives", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : true, + "c2" : 1, + "c3" : 400000000000, + "c4" : 1.284765648, + "c5" : "hello" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : true, + "C2" : 1, + "C3" : 400000000000, + "C4" : 1.284765648, + "C5" : "hello" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n bool c1 = 1;\n int32 c2 = 2;\n int64 c3 = 3;\n double c4 = 4;\n string c5 = 5;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_protobuf_inference_-_primitives/6.0.0_1591141841973/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/plan.json new file mode 100644 index 000000000000..ece53f3b09fc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 STRING) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` STRING" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/spec.json new file mode 100644 index 000000000000..d2c5c114379f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/spec.json @@ -0,0 +1,82 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842026, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should convert enum to STRING", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : "HEARTS" + } + }, { + "topic" : "input", + "key" : "", + "value" : { + "c1" : "SPADES" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : "HEARTS" + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : "SPADES" + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n Suit c1 = 1;\n\n enum Suit {\n SPADES = 0;\n HEARTS = 1;\n DIAMONDS = 2;\n CLUBS = 4;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 STRING" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_enum_to_STRING/6.0.0_1591141842026/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/plan.json new file mode 100644 index 000000000000..8f17dd449998 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 STRUCT) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` STRUCT<`F1` INTEGER>", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` STRUCT<`F1` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` STRUCT<`F1` INTEGER>" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/spec.json new file mode 100644 index 000000000000..d1c784c568a6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/spec.json @@ -0,0 +1,74 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842077, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "should convert nested types to STRUCT", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : { + "f1" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : { + "F1" : 1 + } + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n Something c1 = 1;\n\n message Something {\n int32 f1 = 1;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 STRUCT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_convert_nested_types_to_STRUCT/6.0.0_1591141842077/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/plan.json new file mode 100644 index 000000000000..010702bf13d3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (EXPECTED INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`EXPECTED` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`EXPECTED` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`EXPECTED` INTEGER" + }, + "selectExpressions" : [ "EXPECTED AS EXPECTED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/spec.json new file mode 100644 index 000000000000..7dd9b9323fdb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/spec.json @@ -0,0 +1,71 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842127, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should filter out columns with unsupported types", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "expected" : 1, + "c4" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "EXPECTED" : 1 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n int32 expected = 1;\n bytes c2 = 2;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "EXPECTED INT" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_filter_out_columns_with_unsupported_types/6.0.0_1591141842127/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/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/plan.json new file mode 100644 index 000000000000..60bda6541911 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 ARRAY) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` ARRAY", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`C1` ARRAY" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/spec.json new file mode 100644 index 000000000000..25b3efc8d2de --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/spec.json @@ -0,0 +1,70 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842059, + "path" : "query-validation-tests/protobuf.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "should support arrays", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : [ "a", "", "Bc" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "C1" : [ "a", "", "Bc" ] + } + } ], + "topics" : [ { + "name" : "input", + "schema" : "syntax = \"proto3\";\n\nmessage ConfluentDefault1 {\n repeated string c1 = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "stream", + "schema" : "C1 ARRAY" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/protobuf_-_should_support_arrays/6.0.0_1591141842059/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/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/plan.json new file mode 100644 index 000000000000..4fbc82e9ac93 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/plan.json @@ -0,0 +1,133 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (`old.key` STRING KEY, `some.key` STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`old.key` STRING KEY, `some.key` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM TEST TEST\nPARTITION BY TEST.`some.key`\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`some.key` STRING KEY, `old.key` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "PartitionBy" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`old.key` STRING KEY, `some.key` STRING" + }, + "keyExpression" : "`some.key`" + }, + "keyColumnNames" : [ "some.key" ], + "selectExpressions" : [ "`old.key` AS `old.key`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/spec.json new file mode 100644 index 000000000000..10350bc02db5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/spec.json @@ -0,0 +1,63 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842371, + "path" : "query-validation-tests/quoted-identifiers.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "partition by quoted field", + "inputs" : [ { + "topic" : "test_topic", + "key" : "old-key", + "value" : { + "some.key" : "key" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "key", + "value" : { + "old.key" : "old-key" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (`old.key` VARCHAR KEY, `some.key` VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM TEST PARTITION BY `some.key`;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/topology new file mode 100644 index 000000000000..63f0d28f9a1f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/quoted-identifiers_-_partition_by_quoted_field/6.0.0_1591141842371/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PartitionBy-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: PartitionBy-SelectKey (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- PartitionBy-SelectKey + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/plan.json new file mode 100644 index 000000000000..b2e4b210257c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/spec.json new file mode 100644 index 000000000000..b07afe5ce121 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/spec.json @@ -0,0 +1,84 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842882, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "ARRAY", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous array - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : [ "a", "b", "c" ] + }, { + "topic" : "input_topic", + "key" : "", + "value" : [ "a", "b", null ] + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "array", + "items" : [ "null", "string" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141842882/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/plan.json new file mode 100644 index 000000000000..6193f9584b59 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/spec.json new file mode 100644 index 000000000000..156cf95d2a60 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/spec.json @@ -0,0 +1,84 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842867, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "ARRAY", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous array - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : [ "a", "b", "c" ] + }, { + "topic" : "input_topic", + "key" : "", + "value" : [ "a", "b", null ] + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "array", + "items" : [ "null", "string" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_JSON/6.0.0_1591141842867/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/plan.json new file mode 100644 index 000000000000..b2e4b210257c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/spec.json new file mode 100644 index 000000000000..e2b7b2297819 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/spec.json @@ -0,0 +1,66 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843074, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "ARRAY", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous array - value - non-nullable - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : [ "a", "b", "c" ] + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "array", + "items" : "string" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843074/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/plan.json new file mode 100644 index 000000000000..b2e4b210257c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/spec.json new file mode 100644 index 000000000000..c3c3c901d0a2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/spec.json @@ -0,0 +1,66 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842939, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "ARRAY", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous array - value - with coercion - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : [ 1, 2, 3 ] + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "1", "2", "3" ] + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "array", + "items" : [ "null", "string" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_AVRO/6.0.0_1591141842939/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/plan.json new file mode 100644 index 000000000000..6193f9584b59 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/spec.json new file mode 100644 index 000000000000..3eed7597ed18 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/spec.json @@ -0,0 +1,66 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842899, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "ARRAY", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous array - value - with coercion - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : [ 1, 2, 3 ] + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "1", "2", "3" ] + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "array", + "items" : [ "null", "string" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_array_-_value_-_with_coercion_-_JSON/6.0.0_1591141842899/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/plan.json new file mode 100644 index 000000000000..0df24352fbc6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/spec.json new file mode 100644 index 000000000000..3c657b7cb84a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/spec.json @@ -0,0 +1,129 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843088, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "MAP", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous map - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "map", + "values" : [ "null", "int" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843088/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/plan.json new file mode 100644 index 000000000000..dc88a51331c3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/spec.json new file mode 100644 index 000000000000..57fe458512aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/spec.json @@ -0,0 +1,129 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843104, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "MAP", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous map - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "map", + "values" : [ "null", "int" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843104/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/plan.json new file mode 100644 index 000000000000..0df24352fbc6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/spec.json new file mode 100644 index 000000000000..e05a9a709523 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/spec.json @@ -0,0 +1,79 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843188, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "MAP", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous map - value - non-nullable - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "map", + "values" : "int" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_non-nullable_-_AVRO/6.0.0_1591141843188/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/plan.json new file mode 100644 index 000000000000..46ca4a3bb99b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/spec.json new file mode 100644 index 000000000000..a6054fe0e63d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/spec.json @@ -0,0 +1,103 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843120, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "MAP", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize anonymous map - value - with coercion - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : "1", + "b" : "2", + "c" : "3" + } + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "map", + "values" : [ "null", "int" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_anonymous_map_-_value_-_with_coercion_-_AVRO/6.0.0_1591141843120/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/plan.json new file mode 100644 index 000000000000..39dddd1676cb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=true);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/spec.json new file mode 100644 index 000000000000..6c983ebc2528 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842994, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested array - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=true, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_AVRO/6.0.0_1591141842994/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/plan.json new file mode 100644 index 000000000000..9d385f8eb53f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=true);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/spec.json new file mode 100644 index 000000000000..cb0ef65adb98 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141842979, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested array - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=true, kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_JSON/6.0.0_1591141842979/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/plan.json new file mode 100644 index 000000000000..584461cf281c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='PROTOBUF', WRAP_SINGLE_VALUE=true);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/spec.json new file mode 100644 index 000000000000..430c5d7731f6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/spec.json @@ -0,0 +1,85 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843059, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested array - value - PROTOBUF", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ "a", "b", "c" ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated string FOO = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (WRAP_SINGLE_VALUE=true, kafka_topic='input_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843059/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/plan.json new file mode 100644 index 000000000000..0fe611d8ac45 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO', WRAP_SINGLE_VALUE=true);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/spec.json new file mode 100644 index 000000000000..6269ca76464b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843133, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested map - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "map", + "values" : [ "null", "int" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=true, kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_AVRO/6.0.0_1591141843133/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/plan.json new file mode 100644 index 000000000000..648777d1159f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=true);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/spec.json new file mode 100644 index 000000000000..1a897321b503 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843149, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested map - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "map", + "values" : [ "null", "int" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (WRAP_SINGLE_VALUE=true, kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 1 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_JSON/6.0.0_1591141843149/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/plan.json new file mode 100644 index 000000000000..54d77d9da467 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/spec.json new file mode 100644 index 000000000000..fee9361d1386 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/spec.json @@ -0,0 +1,98 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843166, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "deserialize nested map - value - PROTOBUF", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1, + "b" : 2, + "c" : 3 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated ConnectDefault2Entry FOO = 1;\n\n message ConnectDefault2Entry {\n string key = 1;\n int32 value = 2;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "stream", + "schema" : "FOO MAP" + } ], + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_deserialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843166/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/plan.json new file mode 100644 index 000000000000..dcb87e70eced --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/spec.json new file mode 100644 index 000000000000..c29807538a90 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/spec.json @@ -0,0 +1,109 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843372, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "ARRAY" + }, + "testCase" : { + "name" : "serialize anonymous array - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : [ 12, 34, 999 ] + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : [ 12, 34, null ] + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "array", + "items" : [ "null", "long" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "long" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_AVRO/6.0.0_1591141843372/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/plan.json new file mode 100644 index 000000000000..2b49e509a56f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/spec.json new file mode 100644 index 000000000000..8c2729950eca --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/spec.json @@ -0,0 +1,94 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843352, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "ARRAY" + }, + "testCase" : { + "name" : "serialize anonymous array - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : [ 12, 34, 999 ] + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : [ 12, 34, null ] + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "array", + "items" : [ "null", "long" ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_array_-_value_-_JSON/6.0.0_1591141843352/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/plan.json new file mode 100644 index 000000000000..21feecfc8e6b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/spec.json new file mode 100644 index 000000000000..c99eefe0836a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/spec.json @@ -0,0 +1,151 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843481, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "MAP" + }, + "testCase" : { + "name" : "serialize anonymous map - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ] + } + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_FOO", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_FOO" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_AVRO/6.0.0_1591141843481/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/plan.json new file mode 100644 index 000000000000..1dbfa336085d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/spec.json new file mode 100644 index 000000000000..0f7c53c61d44 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/spec.json @@ -0,0 +1,122 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843462, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "MAP" + }, + "testCase" : { + "name" : "serialize anonymous map - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ] + } + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_map_-_value_-_JSON/6.0.0_1591141843462/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/plan.json new file mode 100644 index 000000000000..8d8024e25266 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRUCT) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` STRUCT<`F0` INTEGER>" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/spec.json new file mode 100644 index 000000000000..c5b8d6f2ef22 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/spec.json @@ -0,0 +1,127 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843604, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT" + }, + "testCase" : { + "name" : "serialize anonymous struct - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "F0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "F0" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "record", + "name" : "KsqlDataSourceSchema_FOO", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_FOO" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo STRUCT) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_AVRO/6.0.0_1591141843604/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/plan.json new file mode 100644 index 000000000000..7b193bd78b46 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/plan.json @@ -0,0 +1,127 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRUCT) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` STRUCT<`F0` INTEGER>" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/spec.json new file mode 100644 index 000000000000..7fea623c25a1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/spec.json @@ -0,0 +1,106 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843583, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT" + }, + "testCase" : { + "name" : "serialize anonymous struct - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "F0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "F0" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo STRUCT) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_anonymous_struct_-_value_-_JSON/6.0.0_1591141843583/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/plan.json new file mode 100644 index 000000000000..37d8bf8a6c9d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/spec.json new file mode 100644 index 000000000000..a1a97b320fb3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/spec.json @@ -0,0 +1,122 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843419, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested array - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "long" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "long" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_AVRO/6.0.0_1591141843419/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/plan.json new file mode 100644 index 000000000000..218efcf1ce4b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/spec.json new file mode 100644 index 000000000000..3cf29d9671c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843396, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested array - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ 12, 34, null ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "long" ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_JSON/6.0.0_1591141843396/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/plan.json new file mode 100644 index 000000000000..e29fd7a39663 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO ARRAY) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`FOO` ARRAY" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/spec.json new file mode 100644 index 000000000000..4f93cbed09f3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/spec.json @@ -0,0 +1,85 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843442, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested array - value - PROTOBUF", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ 12, 34, 999 ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : [ ] + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated int64 FOO = 1;\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo ARRAY) WITH (kafka_topic='input_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_array_-_value_-_PROTOBUF/6.0.0_1591141843442/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/plan.json new file mode 100644 index 000000000000..128477e050a4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/spec.json new file mode 100644 index 000000000000..8b49bb55411e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/spec.json @@ -0,0 +1,164 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843517, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested map - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_FOO", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_FOO" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_AVRO/6.0.0_1591141843517/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/plan.json new file mode 100644 index 000000000000..e68ea7eee55a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/spec.json new file mode 100644 index 000000000000..c77b5ab899a4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/spec.json @@ -0,0 +1,135 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843500, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested map - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "test", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "double" ], + "default" : null + } ] + } + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_JSON/6.0.0_1591141843500/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/plan.json new file mode 100644 index 000000000000..9d3c7aa2252d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO MAP) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` MAP", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`FOO` MAP" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/spec.json new file mode 100644 index 000000000000..dcff803ac9b0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843562, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested map - value - PROTOBUF", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "a" : 1.1, + "b" : 2.2, + "c" : 3.456 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n repeated ConnectDefault2Entry FOO = 1;\n\n message ConnectDefault2Entry {\n string key = 1;\n double value = 2;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo MAP) WITH (kafka_topic='input_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_map_-_value_-_PROTOBUF/6.0.0_1591141843562/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/plan.json new file mode 100644 index 000000000000..ba4d9dff5325 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRUCT) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`FOO` STRUCT<`F0` INTEGER>" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/spec.json new file mode 100644 index 000000000000..af3706bc0a3e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/spec.json @@ -0,0 +1,140 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843660, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested struct - value - AVRO", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "record", + "name" : "ignored2", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ] + } ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "record", + "name" : "KsqlDataSourceSchema_FOO", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_FOO" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo STRUCT) WITH (kafka_topic='input_topic', value_format='AVRO');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_AVRO/6.0.0_1591141843660/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/plan.json new file mode 100644 index 000000000000..c4a35bf6e88d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRUCT) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`FOO` STRUCT<`F0` INTEGER>" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/spec.json new file mode 100644 index 000000000000..2cfb26bbbb64 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/spec.json @@ -0,0 +1,119 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843636, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested struct - value - JSON", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "schema" : { + "type" : "record", + "name" : "ignored", + "fields" : [ { + "name" : "FOO", + "type" : [ "null", { + "type" : "record", + "name" : "ignored2", + "fields" : [ { + "name" : "F0", + "type" : [ "null", "int" ] + } ] + } ] + } ] + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo STRUCT) WITH (kafka_topic='input_topic', value_format='JSON');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_JSON/6.0.0_1591141843636/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/plan.json new file mode 100644 index 000000000000..e8eef4bb414a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRUCT) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRUCT<`F0` INTEGER>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + } + }, + "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_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "sourceSchema" : "`FOO` STRUCT<`F0` INTEGER>" + }, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/spec.json new file mode 100644 index 000000000000..96f1c6fa1bf9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/spec.json @@ -0,0 +1,105 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843733, + "path" : "query-validation-tests/serdes.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT> NOT NULL" + }, + "testCase" : { + "name" : "serialize nested struct - value - PROTOBUF", + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : { + "F0" : null + } + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "input_topic", + "key" : "", + "value" : null + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : 1 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : { + "F0" : 0 + } + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "FOO" : null + } + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "schema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n ConnectDefault2 FOO = 1;\n\n message ConnectDefault2 {\n int32 F0 = 1;\n }\n}\n", + "format" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (foo STRUCT) WITH (kafka_topic='input_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=true) AS SELECT * FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + }, { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "PROTOBUF" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/serdes_-_serialize_nested_struct_-_value_-_PROTOBUF/6.0.0_1591141843733/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> 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/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/plan.json new file mode 100644 index 000000000000..a077e389d55a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/plan.json @@ -0,0 +1,262 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM DATA (USER_ID INTEGER) WITH (KAFKA_TOPIC='data', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "DATA", + "schema" : "`USER_ID` INTEGER", + "topicName" : "data", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE SESSIONS AS SELECT\n DATA.USER_ID USER_ID,\n COUNT(DATA.USER_ID) COUNT\nFROM DATA DATA\nWINDOW SESSION ( 5 SECONDS ) \nGROUP BY DATA.USER_ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "SESSIONS", + "schema" : "`USER_ID` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "SESSIONS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "queryPlan" : { + "sources" : [ "DATA" ], + "sink" : "SESSIONS", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "SESSIONS" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "data", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`USER_ID` INTEGER" + }, + "selectExpressions" : [ "USER_ID AS USER_ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "USER_ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "USER_ID" ], + "aggregationFunctions" : [ "COUNT(USER_ID)" ], + "windowExpression" : " SESSION ( 5 SECONDS ) " + }, + "keyColumnNames" : [ "USER_ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "SESSIONS" + }, + "queryId" : "CTAS_SESSIONS_0" + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SESSION_STREAM (USER_ID INTEGER KEY, COUNT BIGINT) WITH (KAFKA_TOPIC='SESSIONS', VALUE_FORMAT='JSON', WINDOW_TYPE='Session');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SESSION_STREAM", + "schema" : "`USER_ID` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "SESSIONS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "SESSION" + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SESSION_STARTS AS SELECT *\nFROM SESSION_STREAM SESSION_STREAM\nWHERE (SESSION_STREAM.WINDOWSTART = SESSION_STREAM.WINDOWEND)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SESSION_STARTS", + "schema" : "`USER_ID` INTEGER KEY, `COUNT` BIGINT", + "topicName" : "SESSION_STARTS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "queryPlan" : { + "sources" : [ "SESSION_STREAM" ], + "sink" : "SESSION_STARTS", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SESSION_STARTS" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamFilterV1", + "properties" : { + "queryContext" : "WhereFilter" + }, + "source" : { + "@type" : "windowedStreamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "SESSIONS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "windowInfo" : { + "type" : "SESSION" + }, + "sourceSchema" : "`USER_ID` INTEGER KEY, `COUNT` BIGINT" + }, + "filterExpression" : "(WINDOWSTART = WINDOWEND)" + }, + "keyColumnNames" : [ "USER_ID" ], + "selectExpressions" : [ "COUNT AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "SESSION_STARTS" + }, + "queryId" : "CSAS_SESSION_STARTS_1" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/spec.json new file mode 100644 index 000000000000..3f090b0c34ef --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/spec.json @@ -0,0 +1,223 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141843871, + "path" : "query-validation-tests/session-windows.json", + "schemas" : { + "CSAS_SESSION_STARTS_1.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_SESSION_STARTS_1.SESSION_STARTS" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "session start stream", + "inputs" : [ { + "topic" : "data", + "key" : "", + "value" : { + "user_id" : 5 + }, + "timestamp" : 0 + }, { + "topic" : "data", + "key" : "", + "value" : { + "user_id" : 5 + }, + "timestamp" : 1000 + }, { + "topic" : "data", + "key" : "", + "value" : { + "user_id" : 5 + }, + "timestamp" : 2000 + }, { + "topic" : "data", + "key" : "", + "value" : { + "user_id" : 5 + }, + "timestamp" : 70000 + } ], + "outputs" : [ { + "topic" : "SESSIONS", + "key" : 5, + "value" : { + "COUNT" : 1 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 0, + "type" : "SESSION" + } + }, { + "topic" : "SESSIONS", + "key" : 5, + "value" : null, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 0, + "type" : "SESSION" + } + }, { + "topic" : "SESSIONS", + "key" : 5, + "value" : { + "COUNT" : 2 + }, + "timestamp" : 1000, + "window" : { + "start" : 0, + "end" : 1000, + "type" : "SESSION" + } + }, { + "topic" : "SESSIONS", + "key" : 5, + "value" : null, + "timestamp" : 1000, + "window" : { + "start" : 0, + "end" : 1000, + "type" : "SESSION" + } + }, { + "topic" : "SESSIONS", + "key" : 5, + "value" : { + "COUNT" : 3 + }, + "timestamp" : 2000, + "window" : { + "start" : 0, + "end" : 2000, + "type" : "SESSION" + } + }, { + "topic" : "SESSIONS", + "key" : 5, + "value" : { + "COUNT" : 1 + }, + "timestamp" : 70000, + "window" : { + "start" : 70000, + "end" : 70000, + "type" : "SESSION" + } + }, { + "topic" : "SESSION_STARTS", + "key" : 5, + "value" : { + "COUNT" : 1 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 0, + "type" : "SESSION" + } + }, { + "topic" : "SESSION_STARTS", + "key" : 5, + "value" : { + "COUNT" : 1 + }, + "timestamp" : 70000, + "window" : { + "start" : 70000, + "end" : 70000, + "type" : "SESSION" + } + } ], + "topics" : [ { + "name" : "data", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "SESSIONS", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "SESSION_STARTS", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM DATA (USER_ID INT) WITH (kafka_topic='data', value_format='JSON');", "CREATE TABLE SESSIONS as SELECT USER_ID, COUNT(USER_ID) AS COUNT FROM DATA WINDOW SESSION (5 SECONDS) group by USER_ID;", "CREATE STREAM SESSION_STREAM (USER_ID INT KEY, COUNT BIGINT) WITH (kafka_topic='SESSIONS', value_format='JSON', window_type='Session');", "CREATE STREAM SESSION_STARTS AS SELECT * FROM SESSION_STREAM WHERE WINDOWSTART = WINDOWEND;" ], + "post" : { + "sources" : [ { + "name" : "SESSION_STARTS", + "type" : "stream", + "schema" : "USER_ID INT KEY, COUNT BIGINT" + } ], + "topics" : { + "topics" : [ { + "name" : "SESSIONS", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "SESSION_STARTS", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_SESSIONS_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_SESSIONS_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + }, + "windowInfo" : { + "type" : "SESSION" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "data", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/topology new file mode 100644 index 000000000000..bdd496dcbf3f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_session_start_stream/6.0.0_1591141843871/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [SESSIONS]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> WhereFilter + <-- KSTREAM-SOURCE-0000000000 + Processor: WhereFilter (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- WhereFilter + Sink: KSTREAM-SINK-0000000004 (topic: SESSION_STARTS) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/plan.json new file mode 100644 index 000000000000..aec3ded84f6f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (C1 INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S", + "schema" : "`C1` INTEGER", + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` INTEGER" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "S" + }, + "queryId" : "CSAS_S_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/spec.json new file mode 100644 index 000000000000..d3f9015bf4e1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/spec.json @@ -0,0 +1,76 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141844865, + "path" : "query-validation-tests/sink-partitions-replicas.json", + "schemas" : { + "CSAS_S_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_S_0.S" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "Use the legacy default sink properties for the sink topic if default partitions and replicas were set", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "S", + "key" : "", + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "int" + } ] + }, + "format" : "AVRO", + "replicas" : 3, + "numPartitions" : 5 + }, { + "name" : "S", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST WITH (kafka_topic='input', value_format='AVRO');", "CREATE STREAM S as SELECT * FROM test;" ], + "properties" : { + "ksql.sink.partitions" : "4", + "ksql.sink.replicas" : "1" + }, + "post" : { + "topics" : { + "topics" : [ { + "name" : "S", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 5 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/topology new file mode 100644 index 000000000000..8155e3a5e26f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_Use_the_legacy_default_sink_properties_for_the_sink_topic_if_default_partitions_and_replicas_were_set/6.0.0_1591141844865/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: S) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/plan.json new file mode 100644 index 000000000000..aec3ded84f6f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/plan.json @@ -0,0 +1,125 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (C1 INTEGER) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S", + "schema" : "`C1` INTEGER", + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`C1` INTEGER" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "S" + }, + "queryId" : "CSAS_S_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/spec.json new file mode 100644 index 000000000000..f4f7e47cef00 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591141844849, + "path" : "query-validation-tests/sink-partitions-replicas.json", + "schemas" : { + "CSAS_S_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_S_0.S" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "should copy partition and replica count from source topic", + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "S", + "key" : "", + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "schema" : { + "type" : "record", + "name" : "blah", + "fields" : [ { + "name" : "c1", + "type" : "int" + } ] + }, + "format" : "AVRO", + "replicas" : 3, + "numPartitions" : 5 + }, { + "name" : "S", + "replicas" : 3, + "numPartitions" : 5 + } ], + "statements" : [ "CREATE STREAM TEST WITH (kafka_topic='input', value_format='AVRO');", "CREATE STREAM S as SELECT * FROM test;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "S", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 5 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/topology new file mode 100644 index 000000000000..8155e3a5e26f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sink-partitions-replicas_-_should_copy_partition_and_replica_count_from_source_topic/6.0.0_1591141844849/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: S) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/avro.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/avro.json index 0cd743c3fbfd..604420996adb 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/avro.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/avro.json @@ -23,7 +23,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": true, "C2": 1, "C3": 400000000000, "C4": 1.284765648, "C5": "hello"}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} ] } }, @@ -49,7 +49,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": "SPADES"}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 STRING"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 STRING"} ] } }, @@ -72,7 +72,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": ["a", "", "Bc"]}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 ARRAY"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 ARRAY"} ] } }, @@ -95,7 +95,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": {"f1": 1}}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 MAP"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 MAP"} ] } }, @@ -123,7 +123,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": {"F1": 1}}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 STRUCT"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 STRUCT"} ] } },{ @@ -148,7 +148,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": 4.0}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 DOUBLE"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 DOUBLE"} ] } }, @@ -173,7 +173,34 @@ "outputs": [{"topic": "OUTPUT", "value": {"EXPECTED": 1}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, EXPECTED INT"} + {"name": "OUTPUT", "type": "stream", "schema": "EXPECTED INT"} + ] + } + }, + { + "name": "should support partial schemas", + "statements": [ + "CREATE STREAM INPUT (ID INT KEY) WITH (kafka_topic='input', value_format='AvRo');", + "CREATE STREAM OUTPUT AS SELECT * FROM input;" + ], + "topics": [ + { + "name": "input", + "format": "AVRO", + "schema": {"name": "blah", "type": "record", "fields": [ + {"name": "c1", "type": "boolean"}, + {"name": "c2", "type": "int"}, + {"name": "c3", "type": "long"}, + {"name": "c4", "type": "double"}, + {"name": "c5", "type": "string"} + ]} + } + ], + "inputs": [{"topic": "input", "key": 1, "value": {"c1": true, "c2": 1, "c3": 400000000000, "c4": 1.284765648, "c5": "hello"}}], + "outputs": [{"topic": "OUTPUT", "key": 1, "value": {"C1": true, "C2": 1, "C3": 400000000000, "C4": 1.284765648, "C5": "hello"}}], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} ] } } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/create-struct.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/create-struct.json index 7923cff05a66..483b65f457ad 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/create-struct.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/create-struct.json @@ -89,14 +89,14 @@ { "name": "duplicate structs in array", "statements": [ - "CREATE STREAM INPUT (id VARCHAR) WITH (kafka_topic='test',value_format='json',partitions=1);", - "CREATE STREAM OUTPUT AS SELECT ROWKEY id, array[struct(a:=123),struct(a:=123)] from INPUT emit changes;" + "CREATE STREAM INPUT (ignored VARCHAR) WITH (kafka_topic='test',value_format='json',partitions=1);", + "CREATE STREAM OUTPUT AS SELECT array[struct(a:=123),struct(a:=123)] from INPUT emit changes;" ], "inputs": [ { "topic": "test", "value": {"col1": "hello world"}} ], "outputs": [ - {"topic": "OUTPUT","value": {"KSQL_COL_0": [{"A":123}, {"A":123}]}} + {"topic": "OUTPUT", "value": {"KSQL_COL_0": [{"A":123}, {"A":123}]}} ] }, { diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json index 5a89d5fe7d36..cff818292420 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json @@ -101,7 +101,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"@TIMESTAMP": 4, "FROM": 5}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, `@TIMESTAMP` BIGINT, `FROM` BIGINT"} + {"name": "OUTPUT", "type": "stream", "schema": "`@TIMESTAMP` BIGINT, `FROM` BIGINT"} ] } }, @@ -859,6 +859,54 @@ {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, ID_COPY INT"} ] } + }, + { + "name": "table without primary key fails", + "statements": [ + "CREATE TABLE INPUT (ID INT, F0 INT) WITH (kafka_topic='input', value_format='JSON');" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Tables require a PRIMARY KEY. Please define the PRIMARY KEY." + } + }, + { + "name": "table without primary key fails - schema inference", + "statements": [ + "CREATE TABLE INPUT WITH (kafka_topic='input', value_format='Avro');" + ], + "topics": [ + { + "name": "input", + "schema": {"name": "blah", "type": "record", "fields": [{"name": "c1", "type": "int"}]}, + "format": "AVRO" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Tables require a PRIMARY KEY. Please define the PRIMARY KEY.\nUse a partial schema to define the primary key and still load the value columns from the Schema Registry, for example:\n\tCREATE TABLE INPUT (ID INT PRIMARY KEY) WITH (...);" + } + }, + { + "name": "stream without key column", + "statements": [ + "CREATE STREAM INPUT (ID INT, F0 INT) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "inputs": [ + {"topic": "input", "key": null, "value": {"ID": 1, "F0": 2}}, + {"topic": "input", "key": "foo", "value": {"ID": 2, "F0": 4}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": null, "value": {"ID": 1, "F0": 2}}, + {"topic": "OUTPUT", "key": null, "value": {"ID": 2, "F0": 4}} + ], + "post": { + "sources": [ + {"name": "INPUT", "type": "stream", "schema": "ID INT, F0 INT"}, + {"name": "OUTPUT", "type": "stream", "schema": "ID INT, F0 INT"} + ] + } } ] } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json index 615b7b98654f..eee70e082722 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json @@ -288,6 +288,34 @@ ] } }, + { + "name": "steam with no key", + "statements": [ + "CREATE STREAM TEST (data INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE OUTPUT AS SELECT DATA, COUNT(*) AS COUNT FROM TEST GROUP BY DATA;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"data": 22}}, + {"topic": "test_topic", "value": {"data": 333}}, + {"topic": "test_topic", "value": {"data": 22}} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", "key": 22, "value": {"ROWTIME": 0, "DATA": 22}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", "key": 333, "value": {"ROWTIME": 0, "DATA": 333}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", "key": 22, "value": {"ROWTIME": 0, "DATA": 22}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 22, "value": {"ROWTIME": 0, "DATA": 22, "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 333, "value": {"ROWTIME": 0, "DATA": 333, "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 22, "value": {"ROWTIME": 0, "DATA": 22, "KSQL_AGG_VARIABLE_0": 2}}, + {"topic": "OUTPUT", "key": 22, "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": 333, "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": 22, "value": {"COUNT": 2}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "table", "schema": "DATA INT KEY, COUNT BIGINT"} + ] + } + }, { "name": "unknown function", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json index 60d876625e2a..93b51f958ddd 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json @@ -18,6 +18,23 @@ {"topic": "OUTPUT", "key": "k1", "value": "v1"} ] }, + { + "name": "streams with no key columns", + "statements": [ + "CREATE STREAM SOURCE1 (K STRING, data VARCHAR) WITH (kafka_topic='stream-source', value_format='JSON');", + "CREATE STREAM SOURCE2 (K STRING, data VARCHAR) WITH (kafka_topic='insert-source', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM SOURCE1;", + "INSERT INTO OUTPUT SELECT * FROM SOURCE2;" + ], + "inputs": [ + {"topic": "stream-source", "value": {"K": "k1", "data": "v1"}}, + {"topic": "insert-source", "value": {"K": "k2", "data": "v2"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": null, "value": {"K": "k1", "DATA": "v1"}}, + {"topic": "OUTPUT", "key": null, "value": {"K": "k2", "DATA": "v2"}} + ] + }, { "name": "with custom topic name", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json index 19e1e643d10e..0f1667826263 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json @@ -1966,6 +1966,46 @@ "outputs": [ {"topic": "OUTPUT", "key": "user_0", "value": {"IMPRESSION_ID": 24, "URL": "urlA"}, "timestamp": 12} ] + }, + { + "name": "streams with no key columns (stream->stream)", + "statements": [ + "CREATE STREAM L (A INT, B INT, C INT) WITH (kafka_topic='LEFT', value_format='JSON');", + "CREATE STREAM R (A INT, B INT, C INT) WITH (kafka_topic='RIGHT', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM L INNER JOIN R WITHIN 10 SECONDS ON L.A = R.A;" + ], + "inputs": [ + {"topic": "LEFT", "value": {"A": 0, "B": 1, "C": 2}, "timestamp": 10}, + {"topic": "RIGHT", "value": {"A": 0, "B": -1, "C": -2}, "timestamp": 11} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"R_A": 0, "L_B": 1, "R_B": -1, "L_C": 2, "R_C": -2}, "timestamp": 11} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "L_A INT KEY, L_B INT, L_C INT, R_A INT, R_B INT, R_C INT"} + ] + } + }, + { + "name": "streams with no key columns (stream->table)", + "statements": [ + "CREATE STREAM L (A INT, B INT, C INT) WITH (kafka_topic='LEFT', value_format='JSON');", + "CREATE TABLE R (A INT PRIMARY KEY, B INT, C INT) WITH (kafka_topic='RIGHT', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM L INNER JOIN R ON L.A = R.A;" + ], + "inputs": [ + {"topic": "RIGHT", "key": 0, "value": {"B": -1, "C": -2}, "timestamp": 10}, + {"topic": "LEFT", "key": "ignored", "value": {"A": 0, "B": 1, "C": 2}, "timestamp": 11} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"R_A": 0, "L_B": 1, "R_B": -1, "L_C": 2, "R_C": -2}, "timestamp": 11} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "L_A INT KEY, L_B INT, L_C INT, R_A INT, R_B INT, R_C INT"} + ] + } } ] } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/partition-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/partition-by.json index 63b1f1a4c4d3..156ea6c10f54 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/partition-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/partition-by.json @@ -34,6 +34,20 @@ ] } }, + { + "name": "no key column", + "statements": [ + "CREATE STREAM INPUT (NAME STRING, ID INT) with (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS select ID, NAME from INPUT partition by ID;" + ], + "inputs": [{"topic": "input", "value": {"NAME": "bob", "ID": 10}}], + "outputs": [{"topic": "OUTPUT", "key": 10, "value": {"NAME": "bob"}}], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] + } + }, { "name": "single value column - select star", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/protobuf.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/protobuf.json index 5852330fc745..64f1ebed3bc1 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/protobuf.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/protobuf.json @@ -86,7 +86,32 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} + ] + } + }, + { + "name": "protobuf inference - partital schema", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY) WITH (kafka_topic='input', value_format='PROTOBUF');", + "CREATE STREAM OUTPUT AS SELECT * FROM input;" + ], + "topics": [ + { + "name": "input", + "format": "PROTOBUF", + "schema": "syntax = \"proto3\"; message ConfluentDefault1 {bool c1 = 1; int32 c2 = 2; int64 c3 = 3; double c4 = 4; string c5 = 5;}" + } + ], + "inputs": [ + {"topic": "input", "key": "a", "value": {"c1": true, "c2": 1, "c3": 400000000000, "c4": 1.284765648, "c5": "hello"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "a", "value": {"C1": true, "C2": 1, "C3": 400000000000, "C4": 1.284765648, "C5": "hello"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "K STRING KEY, C1 BOOLEAN, C2 INT, C3 BIGINT, C4 DOUBLE, C5 STRING"} ] } }, @@ -113,7 +138,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 STRING"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 STRING"} ] } }, @@ -134,7 +159,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": ["a", "", "Bc"]}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 ARRAY"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 ARRAY"} ] } }, @@ -155,7 +180,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"C1": {"F1": 1}}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 STRUCT"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 STRUCT"} ] } }, @@ -184,7 +209,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, C1 DOUBLE, C2 BIGINT, C3 BIGINT, C4 INT, C5 BIGINT"} + {"name": "OUTPUT", "type": "stream", "schema": "C1 DOUBLE, C2 BIGINT, C3 BIGINT, C4 INT, C5 BIGINT"} ] } }, @@ -205,7 +230,7 @@ "outputs": [{"topic": "OUTPUT", "value": {"EXPECTED": 1}}], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, EXPECTED INT"} + {"name": "OUTPUT", "type": "stream", "schema": "EXPECTED INT"} ] } } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/quoted-identifiers.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/quoted-identifiers.json index c2833f886c45..98ceae8cc44a 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/quoted-identifiers.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/quoted-identifiers.json @@ -71,14 +71,14 @@ { "name": "partition by quoted field", "statements": [ - "CREATE STREAM TEST (`some.key` VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (`old.key` VARCHAR KEY, `some.key` VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT * FROM TEST PARTITION BY `some.key`;" ], "inputs": [ - {"topic": "test_topic", "key": "old-key","value": {"some.key": "key"}} + {"topic": "test_topic", "key": "old-key", "value": {"some.key": "key"}} ], "outputs": [ - {"topic": "OUTPUT", "key": "key", "value": {"ROWKEY": "old-key"}} + {"topic": "OUTPUT", "key": "key", "value": {"old.key": "old-key"}} ] }, { diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/serdes.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/serdes.json index bcf903404345..ebae9b25b6d3 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/serdes.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/serdes.json @@ -247,11 +247,7 @@ ], "post": { "sources": [ - { - "name": "INPUT", - "type": "stream", - "schema": "ROWKEY STRING KEY, FOO MAP" - } + {"name": "INPUT", "type": "stream", "schema": "FOO MAP"} ] } }, @@ -281,11 +277,7 @@ ], "post": { "sources": [ - { - "name": "INPUT", - "type": "stream", - "schema": "ROWKEY STRING KEY, FOO MAP" - } + {"name": "INPUT", "type": "stream", "schema": "FOO MAP"} ] } }, @@ -320,11 +312,7 @@ ], "post": { "sources": [ - { - "name": "INPUT", - "type": "stream", - "schema": "ROWKEY STRING KEY, FOO MAP" - } + {"name": "INPUT", "type": "stream", "schema": "FOO MAP"} ] } }, @@ -346,11 +334,7 @@ ], "post": { "sources": [ - { - "name": "INPUT", - "type": "stream", - "schema": "ROWKEY STRING KEY, FOO MAP" - } + {"name": "INPUT", "type": "stream", "schema": "FOO MAP"} ] } }, @@ -375,11 +359,7 @@ ], "post": { "sources": [ - { - "name": "INPUT", - "type": "stream", - "schema": "ROWKEY STRING KEY, FOO MAP" - } + {"name": "INPUT", "type": "stream", "schema": "FOO MAP"} ] } }, diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java index 1340d1bdf6a4..38d9718ad7a6 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java @@ -21,10 +21,8 @@ public class KsqlTypesDeserializationModule extends SimpleModule { - public KsqlTypesDeserializationModule( - final boolean withImplicitColumns - ) { - addDeserializer(LogicalSchema.class, new LogicalSchemaDeserializer(withImplicitColumns)); + public KsqlTypesDeserializationModule() { + addDeserializer(LogicalSchema.class, new LogicalSchemaDeserializer()); addDeserializer(SqlType.class, new SqlTypeDeserializer()); } } diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializer.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializer.java index 21ea5a9a8a2d..1fbfedb6a5dd 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializer.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializer.java @@ -25,11 +25,6 @@ import java.io.IOException; final class LogicalSchemaDeserializer extends JsonDeserializer { - final boolean withImplicitColumns; - - LogicalSchemaDeserializer(final boolean withImplicitColumns) { - this.withImplicitColumns = withImplicitColumns; - } @Override public LogicalSchema deserialize( @@ -41,6 +36,6 @@ public LogicalSchema deserialize( final TableElements tableElements = SchemaParser.parse(text, TypeRegistry.EMPTY); - return tableElements.toLogicalSchema(withImplicitColumns); + return tableElements.toLogicalSchema(); } } diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java index 95fedd6acbbc..a0aa941ef608 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/TableElements.java @@ -20,9 +20,7 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.LogicalSchema.Builder; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlType; -import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; import java.util.Iterator; import java.util.List; @@ -77,24 +75,15 @@ public String toString() { } /** - * @param withImplicitColumns controls if schema has implicit columns such as ROWTIME or ROWKEY. * @return the logical schema. */ - public LogicalSchema toLogicalSchema(final boolean withImplicitColumns) { + public LogicalSchema toLogicalSchema() { if (elements.isEmpty()) { throw new KsqlException("No columns supplied."); } final Builder builder = LogicalSchema.builder(); - if (withImplicitColumns) { - - final boolean noKey = elements.stream().noneMatch(e -> e.getNamespace().isKey()); - if (noKey) { - builder.keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING); - } - } - for (final TableElement tableElement : this) { final ColumnName fieldName = tableElement.getName(); final SqlType fieldType = tableElement.getType().getSqlType(); diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/KsqlTypesSerdeModuleTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/KsqlTypesSerdeModuleTest.java index bd738921a44e..4a1ba9c2b859 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/KsqlTypesSerdeModuleTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/KsqlTypesSerdeModuleTest.java @@ -31,7 +31,7 @@ public class KsqlTypesSerdeModuleTest { private static final ObjectMapper MAPPER = new ObjectMapper() - .registerModule(new KsqlTypesDeserializationModule(false)) + .registerModule(new KsqlTypesDeserializationModule()) .registerModule(new KsqlTypesSerializationModule()); @Test diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializerTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializerTest.java index a6d92d42c1eb..755bae62d098 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializerTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/json/LogicalSchemaDeserializerTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import org.junit.BeforeClass; import org.junit.Test; @@ -33,7 +32,7 @@ public class LogicalSchemaDeserializerTest { @BeforeClass public static void classSetUp() { - MAPPER.registerModule(new TestModule(false)); + MAPPER.registerModule(new TestModule()); } @Test @@ -81,29 +80,12 @@ public void shouldDeserializeSchemaWithKeyAfterValue() throws Exception { .build())); } - @Test - public void shouldAddImplicitColumns() throws Exception { - // Given: - final ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new TestModule(true)); - final String json = "\"`v0` INTEGER\""; - - // When: - final LogicalSchema schema = mapper.readValue(json, LogicalSchema.class); - - // Then: - assertThat(schema, is(LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) - .build())); - } - private static class TestModule extends SimpleModule { - private TestModule(final boolean withImplicitColumns) { + private TestModule() { addDeserializer( LogicalSchema.class, - new LogicalSchemaDeserializer(withImplicitColumns) + new LogicalSchemaDeserializer() ); } } diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java index a6bcde062013..33d8b627e2bc 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java @@ -32,7 +32,6 @@ 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.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; import java.util.List; @@ -213,7 +212,7 @@ public void shouldThrowWhenBuildLogicalSchemaIfNoElements() { // When: final Exception e = assertThrows( KsqlException.class, - () -> tableElements.toLogicalSchema(true) + () -> tableElements.toLogicalSchema() ); // Then: @@ -222,25 +221,24 @@ public void shouldThrowWhenBuildLogicalSchemaIfNoElements() { } @Test - public void shouldBuildLogicalSchemaWithImplicits() { + public void shouldBuildLogicalSchemaWithOutKey() { // Given: final TableElements tableElements = TableElements.of( tableElement(VALUE, "v0", INT_TYPE) ); // When: - final LogicalSchema schema = tableElements.toLogicalSchema(true); + final LogicalSchema schema = tableElements.toLogicalSchema(); // Then: assertThat(schema, is(LogicalSchema.builder() - .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) .build() )); } @Test - public void shouldBuildLogicalSchemaWithImplicitsAndExplicitKey() { + public void shouldBuildLogicalSchemaWithWithKey() { // Given: final TableElements tableElements = TableElements.of( tableElement(VALUE, "v0", INT_TYPE), @@ -248,7 +246,7 @@ public void shouldBuildLogicalSchemaWithImplicitsAndExplicitKey() { ); // When: - final LogicalSchema schema = tableElements.toLogicalSchema(true); + final LogicalSchema schema = tableElements.toLogicalSchema(); // Then: assertThat(schema, is(LogicalSchema.builder() @@ -259,7 +257,7 @@ public void shouldBuildLogicalSchemaWithImplicitsAndExplicitKey() { } @Test - public void shouldBuildLogicalSchemaWithImplicitsAndExplicitPrimaryKey() { + public void shouldBuildLogicalSchemaWithWithPrimaryKey() { // Given: final TableElements tableElements = TableElements.of( tableElement(VALUE, "v0", INT_TYPE), @@ -267,43 +265,7 @@ public void shouldBuildLogicalSchemaWithImplicitsAndExplicitPrimaryKey() { ); // When: - final LogicalSchema schema = tableElements.toLogicalSchema(true); - - // Then: - assertThat(schema, is(LogicalSchema.builder() - .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) - .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) - .build() - )); - } - - @Test - public void shouldBuildLogicalSchemaWithOutImplicits() { - // Given: - final TableElements tableElements = TableElements.of( - tableElement(VALUE, "v0", INT_TYPE) - ); - - // When: - final LogicalSchema schema = tableElements.toLogicalSchema(false); - - // Then: - assertThat(schema, is(LogicalSchema.builder() - .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) - .build() - )); - } - - @Test - public void shouldBuildLogicalSchemaWithOutImplicitsWithPrimaryKey() { - // Given: - final TableElements tableElements = TableElements.of( - tableElement(VALUE, "v0", INT_TYPE), - tableElement(PRIMARY_KEY, "k0", INT_TYPE) - ); - - // When: - final LogicalSchema schema = tableElements.toLogicalSchema(false); + final LogicalSchema schema = tableElements.toLogicalSchema(); // Then: assertThat(schema, is(LogicalSchema.builder() @@ -324,7 +286,7 @@ public void shouldBuildLogicalSchemaWithKeyAndValueColumnsInterleaved() { ); // When: - final LogicalSchema schema = tableElements.toLogicalSchema(false); + final LogicalSchema schema = tableElements.toLogicalSchema(); // Then: assertThat(schema, is(LogicalSchema.builder() diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java index 0e244770af22..0b039dceeeb2 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java @@ -133,10 +133,10 @@ public void tearDown() throws Exception { public void shouldHandleJsonWithSchemas() { // Given: givenScript("" - + "CREATE STREAM S (ORDERTIME BIGINT)" + + "CREATE STREAM S (ROWKEY STRING KEY, ORDERTIME BIGINT)" + " WITH (kafka_topic='" + JSON_TOPIC + "', value_format='json');\n" + "\n" - + "CREATE TABLE T (ORDERTIME BIGINT) " + + "CREATE TABLE T (ROWKEY STRING PRIMARY KEY, ORDERTIME BIGINT) " + " WITH (kafka_topic='" + JSON_TOPIC + "', value_format='json');\n" + "\n" + "SET 'auto.offset.reset' = 'earliest';" @@ -175,10 +175,10 @@ public void shouldHandleJsonWithSchemas() { public void shouldHandleAvroWithSchemas() { // Given: givenScript("" - + "CREATE STREAM S (ORDERTIME BIGINT)" + + "CREATE STREAM S (ROWKEY STRING KEY, ORDERTIME BIGINT)" + " WITH (kafka_topic='" + AVRO_TOPIC + "', value_format='avro');\n" + "\n" - + "CREATE TABLE T (ORDERTIME BIGINT) " + + "CREATE TABLE T (ROWKEY STRING PRIMARY KEY, ORDERTIME BIGINT) " + " WITH (kafka_topic='" + AVRO_TOPIC + "', value_format='avro');\n" + "\n" + "SET 'auto.offset.reset' = 'earliest';" @@ -219,7 +219,7 @@ public void shouldInferAvroSchema() { givenScript("" + "SET 'auto.offset.reset' = 'earliest';" + "" - + "CREATE STREAM S WITH (kafka_topic='" + AVRO_TOPIC + "', value_format='avro');\n" + + "CREATE STREAM S (ROWKEY STRING KEY) WITH (kafka_topic='" + AVRO_TOPIC + "', value_format='avro');\n" + "" + "CREATE STREAM " + s1 + " AS SELECT * FROM S;"); @@ -285,7 +285,7 @@ public void shouldHandleComments() { + "" + "SET 'auto.offset.reset' = 'earliest';" + "" - + "CREATE STREAM S /*inline comment*/ (ID int)" + + "CREATE STREAM S /*inline comment*/ (ROWKEY STRING KEY, ID int)" + " with (kafka_topic='" + JSON_TOPIC + "',value_format='json');\n" + "\n" + "CREATE STREAM " + s1 + " AS SELECT * FROM S;"); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 16827c0e4505..65a9f29b7d99 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -566,7 +566,7 @@ public void shouldRecoverCreates() { @Test public void shouldRecoverRecreates() { server1.submitCommands( - "CREATE STREAM A (C1 STRING, C2 INT) WITH (KAFKA_TOPIC='A', VALUE_FORMAT='JSON');", + "CREATE STREAM A (ROWKEY STRING KEY, C1 STRING, C2 INT) WITH (KAFKA_TOPIC='A', VALUE_FORMAT='JSON');", "CREATE STREAM B AS SELECT ROWKEY, C1 FROM A;", "TERMINATE CsAs_b_0;", "DROP STREAM B;", diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index f866dcb661ba..d20b53fefed7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -1864,13 +1864,13 @@ public void shouldFailIfCreateExistingSourceTable() { givenSource(DataSourceType.KTABLE, "SOURCE", "topic1", SINGLE_FIELD_SCHEMA); givenKafkaTopicExists("topic2"); final String createSql = - "CREATE TABLE SOURCE (val int) WITH (kafka_topic='topic2', value_format='json');"; + "CREATE TABLE SOURCE (id int primary key, val int) WITH (kafka_topic='topic2', value_format='json');"; // When: final KsqlRestException e = assertThrows( KsqlRestException.class, () -> makeSingleRequest(createSql, CommandStatusEntity.class) -); + ); // Then: assertThat(e, exceptionStatusCode(is(BAD_REQUEST.code()))); diff --git a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java index 38f677fb1490..77b41c47cc57 100644 --- a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java +++ b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java @@ -36,7 +36,7 @@ public final class KsqlClient implements AutoCloseable { static { - ApiJsonMapper.INSTANCE.get().registerModule(new KsqlTypesDeserializationModule(false)); + ApiJsonMapper.INSTANCE.get().registerModule(new KsqlTypesDeserializationModule()); } private final Vertx vertx; diff --git a/ksqldb-rest-client/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityTest.java b/ksqldb-rest-client/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityTest.java index b899149b459e..7d0510c7fcc6 100644 --- a/ksqldb-rest-client/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityTest.java +++ b/ksqldb-rest-client/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityTest.java @@ -52,7 +52,7 @@ public class TableRowsEntityTest { MAPPER = new ObjectMapper(); MAPPER.registerModule(new Jdk8Module()); MAPPER.registerModule(new KsqlTypesSerializationModule()); - MAPPER.registerModule(new KsqlTypesDeserializationModule(false)); + MAPPER.registerModule(new KsqlTypesDeserializationModule()); } @Test(expected = IllegalArgumentException.class) diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java index 4ca72c1854a9..3f4e3895a0ec 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java @@ -29,6 +29,7 @@ import io.confluent.ksql.util.KsqlException; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.function.Supplier; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; @@ -85,6 +86,11 @@ public static Serde getPrimitiveSerde(final ConnectSchema schema) { } final List fields = schema.fields(); + if (fields.isEmpty()) { + // No columns: + return (Serde) Serdes.Void(); + } + if (fields.size() != 1) { final String got = new SqlSchemaFormatter(w -> false, Option.AS_COLUMN_LIST).format(schema); throw new KsqlException("The '" + FormatFactory.KAFKA.name() @@ -108,16 +114,21 @@ public static Serde getPrimitiveSerde(final ConnectSchema schema) { private static final class RowSerializer implements Serializer { private final Serializer delegate; - private final Field field; + private final Optional field; RowSerializer(final Serializer delegate, final ConnectSchema schema) { this.delegate = Objects.requireNonNull(delegate, "delegate"); - this.field = schema.fields().get(0); + this.field = schema.fields().isEmpty() + ? Optional.empty() + : Optional.of(schema.fields().get(0)); } @Override public byte[] serialize(final String topic, final Object struct) { - final Object value = struct == null ? null : ((Struct) struct).get(field); + final Object value = struct == null + ? null + : ((Struct) struct).get(field.orElseThrow(IllegalStateException::new)); + return delegate.serialize(topic, value); } } @@ -126,7 +137,7 @@ private static final class RowDeserializer implements Deserializer { private final Deserializer delegate; private final ConnectSchema schema; - private final Field field; + private final Optional field; RowDeserializer( final Deserializer delegate, @@ -134,7 +145,9 @@ private static final class RowDeserializer implements Deserializer { ) { this.delegate = Objects.requireNonNull(delegate, "delegate"); this.schema = Objects.requireNonNull(schema, "schema"); - this.field = schema.fields().get(0); + this.field = schema.fields().isEmpty() + ? Optional.empty() + : Optional.of(schema.fields().get(0)); } @Override @@ -144,8 +157,9 @@ public Struct deserialize(final String topic, final byte[] bytes) { if (primitive == null) { return null; } + final Struct struct = new Struct(schema); - struct.put(field, primitive); + struct.put(field.orElseThrow(IllegalStateException::new), primitive); return struct; } catch (final Exception e) { throw new SerializationException( diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java index 47c59ebc6364..320467f2e58d 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java @@ -180,6 +180,27 @@ public void shouldDeserializeNullAsNull() { assertThat(result, is(nullValue())); } + + @Test + public void shouldHandleNoKeyColumn() { + // Given: + final LogicalSchema logical = LogicalSchema.builder() + .valueColumn(ColumnName.of("f0"), SqlTypes.INTEGER) + .build(); + + final PersistenceSchema schema = PhysicalSchema.from(logical, SerdeOption.none()).keySchema(); + + final Serde serde = factory.createSerde(schema, ksqlConfig, srClientFactory); + + // Given: + final byte[] bytes = serde.serializer().serialize("topic", null); + final Object result = serde.deserializer().deserialize("topic", null); + + // Then: + assertThat(bytes, is(nullValue())); + assertThat(result, is(nullValue())); + } + @Test public void shouldHandleInt() { shouldHandle(SqlTypes.INTEGER, 1); diff --git a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java index db805e92602d..4c6349d80470 100644 --- a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java +++ b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java @@ -47,6 +47,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.Topology.AutoOffsetReset; @@ -346,12 +347,18 @@ private static Consumed buildSourceConsumed( return consumed.withOffsetResetPolicy(getAutoOffsetReset(defaultReset, queryBuilder)); } - private static org.apache.kafka.connect.data.Field getKeySchemaSingleField( - final LogicalSchema schema) { + private static Optional getKeySchemaSingleField( + final LogicalSchema schema + ) { + if (schema.key().isEmpty()) { + return Optional.empty(); + } + if (schema.keyConnectSchema().fields().size() != 1) { throw new IllegalStateException("Only single key fields are currently supported"); } - return schema.keyConnectSchema().fields().get(0); + + return Optional.of(schema.keyConnectSchema().fields().get(0)); } private static String tableChangeLogOpName(final ExecutionStepPropertiesV1 props) { @@ -366,7 +373,7 @@ private static String tableChangeLogOpName(final ExecutionStepPropertiesV1 props private static Function, Collection> windowedKeyGenerator( final LogicalSchema schema ) { - final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); + final Optional keyField = getKeySchemaSingleField(schema); return windowedKey -> { if (windowedKey == null) { @@ -374,7 +381,7 @@ private static Function, Collection> windowedKeyGenerator( } final Window window = windowedKey.window(); - final Object key = windowedKey.key().get(keyField); + final Object key = windowedKey.key().get(keyField.orElseThrow(IllegalStateException::new)); return Arrays.asList(key, window.start(), window.end()); }; } @@ -382,13 +389,13 @@ private static Function, Collection> windowedKeyGenerator( private static Function> nonWindowedKeyGenerator( final LogicalSchema schema ) { - final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); + final Optional keyField = getKeySchemaSingleField(schema); return key -> { if (key == null) { return Collections.singletonList(null); } - return Collections.singletonList(key.get(keyField)); + return Collections.singletonList(key.get(keyField.orElseThrow(IllegalStateException::new))); }; }