diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/name/ColumnNames.java b/ksqldb-common/src/main/java/io/confluent/ksql/name/ColumnNames.java index dd88e59919c6..4ca9d04e90ee 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/name/ColumnNames.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/name/ColumnNames.java @@ -15,11 +15,20 @@ package io.confluent.ksql.name; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.util.KsqlException; +import java.util.List; import java.util.OptionalInt; +import java.util.Set; +import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; public final class ColumnNames { @@ -41,11 +50,24 @@ public static ColumnName aggregateColumn(final int idx) { } /** - * Where the user hasn't specified an alias for an expression in a SELECT we generate them using - * this method. This value is exposed to the user in the output schema + * Create a generator that will build column aliases in the form {@code KSQL_COL_x}. + * + *

Names are guaranteed not to clash with any existing columns in the {@code sourceSchemas}. + * + *

Used where the user hasn't specified an alias for an expression in a SELECT. This generated + * column names are exposed to the user in the output schema. + * + * @param sourceSchemas the stream of source schemas. + * @return a generator of unique column names. */ - public static ColumnName generatedColumnAlias(final int idx) { - return ColumnName.of(GENERATED_ALIAS_PREFIX + idx); + public static Supplier columnAliasGenerator( + final Stream sourceSchemas + ) { + final Set used = generatedAliasIndexes(sourceSchemas) + .boxed() + .collect(Collectors.toSet()); + + return new AliasGenerator(0, used)::next; } /** @@ -71,41 +93,52 @@ public static boolean isAggregate(final ColumnName name) { return name.text().startsWith(AGGREGATE_COLUMN_PREFIX); } - /** - * Determines the next unique column alias. - * - *

Finds any existing {@code KSQL_COL_x} column names in the supplied {@code sourceSchema} to - * ensure the returned generated column name is unique. - * - * @param sourceSchema the source schema. - * @return a column name in the form {@code KSQL_COL_x} which does not clash with source schema. - */ - public static ColumnName nextGeneratedColumnAlias(final LogicalSchema sourceSchema) { - final int maxExistingIdx = maxGeneratedAliasIndex(sourceSchema); - return generatedColumnAlias(maxExistingIdx + 1); + private static OptionalInt extractGeneratedAliasIndex(final ColumnName columnName) { + final Matcher matcher = GENERATED_ALIAS_PATTERN.matcher(columnName.text()); + return matcher.matches() + ? OptionalInt.of(Integer.parseInt(matcher.group(1))) + : OptionalInt.empty(); } - /** - * Determines the highest index of generated column names like {@code KSQL_COL_x} in the supplied - * {@code sourceSchema}. - * - * @param sourceSchema the schema. - * @return the highest index or {@code -1} - */ - private static int maxGeneratedAliasIndex(final LogicalSchema sourceSchema) { - return sourceSchema.columns().stream() + private static IntStream generatedAliasIndexes(final Stream sourceSchema) { + return sourceSchema + .map(LogicalSchema::columns) + .flatMap(List::stream) .map(Column::name) .map(ColumnNames::extractGeneratedAliasIndex) .filter(OptionalInt::isPresent) - .mapToInt(OptionalInt::getAsInt) - .max() - .orElse(-1); + .mapToInt(OptionalInt::getAsInt); } - private static OptionalInt extractGeneratedAliasIndex(final ColumnName columnName) { - final Matcher matcher = GENERATED_ALIAS_PATTERN.matcher(columnName.text()); - return matcher.matches() - ? OptionalInt.of(Integer.parseInt(matcher.group(1))) - : OptionalInt.empty(); + @VisibleForTesting + static final class AliasGenerator { + + private final Set used; + private int next; + + AliasGenerator(final int initial, final Set used) { + this.used = ImmutableSet.copyOf(used); + this.next = initial; + } + + ColumnName next() { + return ColumnName.of(GENERATED_ALIAS_PREFIX + nextIndex()); + } + + private int nextIndex() { + int idx; + + do { + idx = next++; + + if (idx < 0) { + throw new KsqlException("Wow, you've managed to use up all possible generated aliases. " + + "Impressive! Please provide explicit aliases to some of your columns"); + } + + } while (used.contains(idx)); + + return idx; + } } } diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/name/ColumnNamesTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/name/ColumnNamesTest.java index 89aa4d757871..57b33769a1b5 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/name/ColumnNamesTest.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/name/ColumnNamesTest.java @@ -16,56 +16,75 @@ package io.confluent.ksql.name; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; +import com.google.common.collect.ImmutableSet; +import io.confluent.ksql.name.ColumnNames.AliasGenerator; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.util.KsqlException; +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.junit.Test; public class ColumnNamesTest { @Test - public void shouldGenerateUniqueAliasesStartingAtZero() { + public void shouldStartGeneratingFromZeroIfSourceSchemasHaveNoGeneratedAliases() { // Given: - final LogicalSchema schema = LogicalSchema.builder() - .build(); + final Supplier generator = ColumnNames + .columnAliasGenerator(Stream.of(LogicalSchema.builder().build())); // When: - final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema); + final ColumnName result = generator.get(); // Then: assertThat(result, is(ColumnName.of("KSQL_COL_0"))); } @Test - public void shouldGenerateUniqueAliasesTakingAnyKeyColumnsIntoAccount() { + public void shouldAvoidClashesWithSourceColumnNames() { // Given: - final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(ColumnName.of("Fred"), SqlTypes.STRING) - .keyColumn(ColumnNames.generatedColumnAlias(1), SqlTypes.STRING) - .keyColumn(ColumnName.of("George"), SqlTypes.STRING) - .build(); + final Supplier generator = ColumnNames + .columnAliasGenerator(Stream.of(LogicalSchema.builder() + .keyColumn(ColumnName.of("Fred"), SqlTypes.STRING) + .keyColumn(ColumnName.of("KSQL_COL_3"), SqlTypes.STRING) + .keyColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.STRING) + .valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.STRING) + .valueColumn(ColumnName.of("George"), SqlTypes.STRING) + .valueColumn(ColumnName.of("KSQL_COL_5"), SqlTypes.STRING) + .build() + )); // When: - final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema); + final List result = IntStream.range(0, 5) + .mapToObj(idx -> generator.get()) + .collect(Collectors.toList()); // Then: - assertThat(result, is(ColumnName.of("KSQL_COL_2"))); + assertThat(result, contains( + ColumnName.of("KSQL_COL_0"), + ColumnName.of("KSQL_COL_2"), + ColumnName.of("KSQL_COL_4"), + ColumnName.of("KSQL_COL_6"), + ColumnName.of("KSQL_COL_7") + )); } @Test - public void shouldGenerateUniqueAliasesTakingAnyValueColumnsIntoAccount() { + public void shouldThrowIfIndexOverflows() { // Given: - final LogicalSchema schema = LogicalSchema.builder() - .valueColumn(ColumnName.of("Fred"), SqlTypes.STRING) - .valueColumn(ColumnNames.generatedColumnAlias(1), SqlTypes.STRING) - .valueColumn(ColumnName.of("George"), SqlTypes.STRING) - .build(); + final AliasGenerator generator = + new AliasGenerator(Integer.MAX_VALUE, ImmutableSet.of()); - // When: - final ColumnName result = ColumnNames.nextGeneratedColumnAlias(schema); + generator.next(); // returns MAX_VALUE. - // Then: - assertThat(result, is(ColumnName.of("KSQL_COL_2"))); + // When: + assertThrows(KsqlException.class, generator::next); } } \ No newline at end of file diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/AstSanitizer.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/AstSanitizer.java index ad79cef5ccea..ae93626134d6 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/AstSanitizer.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/AstSanitizer.java @@ -15,6 +15,8 @@ package io.confluent.ksql.engine.rewrite; +import static java.util.Objects.requireNonNull; + import io.confluent.ksql.engine.rewrite.ExpressionTreeRewriter.Context; import io.confluent.ksql.execution.expression.tree.DereferenceExpression; import io.confluent.ksql.execution.expression.tree.Expression; @@ -38,9 +40,9 @@ import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.util.KsqlConstants; import io.confluent.ksql.util.KsqlException; -import java.util.Objects; import java.util.Optional; import java.util.function.BiFunction; +import java.util.function.Supplier; /** * Validate and clean ASTs generated from externally supplied statements @@ -69,13 +71,16 @@ private static final class RewriterPlugin extends final MetaStore metaStore; final DataSourceExtractor dataSourceExtractor; - private int selectItemIndex = 0; + private Supplier aliasGenerator; RewriterPlugin(final MetaStore metaStore, final DataSourceExtractor dataSourceExtractor) { super(Optional.empty()); - this.metaStore = Objects.requireNonNull(metaStore, "metaStore"); - this.dataSourceExtractor - = Objects.requireNonNull(dataSourceExtractor, "dataSourceExtractor"); + this.metaStore = requireNonNull(metaStore, "metaStore"); + this.dataSourceExtractor = requireNonNull(dataSourceExtractor, "dataSourceExtractor"); + this.aliasGenerator = ColumnNames.columnAliasGenerator( + dataSourceExtractor.getAllSources().stream() + .map(DataSource::getSchema) + ); } @Override @@ -128,16 +133,16 @@ protected Optional visitSingleColumn( final StatementRewriter.Context ctx ) { if (singleColumn.getAlias().isPresent()) { - selectItemIndex++; return Optional.empty(); } + final ColumnName alias; final Expression expression = ctx.process(singleColumn.getExpression()); if (expression instanceof QualifiedColumnReferenceExp) { final SourceName source = ((QualifiedColumnReferenceExp) expression).getQualifier(); final ColumnName name = ((QualifiedColumnReferenceExp) expression).getColumnName(); if (dataSourceExtractor.isJoin() - && dataSourceExtractor.getCommonFieldNames().contains(name)) { + && dataSourceExtractor.getCommonColumnNames().contains(name)) { alias = ColumnNames.generatedJoinColumnAlias(source, name); } else { alias = name; @@ -152,9 +157,9 @@ protected Optional visitSingleColumn( dereferenceExpressionString.substring( dereferenceExpressionString.indexOf(KsqlConstants.DOT) + 1))); } else { - alias = ColumnNames.generatedColumnAlias(selectItemIndex); + alias = aliasGenerator.get(); } - selectItemIndex++; + return Optional.of( new SingleColumn(singleColumn.getLocation(), expression, Optional.of(alias)) ); @@ -188,9 +193,8 @@ private static final class ExpressionRewriterPlugin extends final MetaStore metaStore, final DataSourceExtractor dataSourceExtractor) { super(Optional.empty()); - this.metaStore = Objects.requireNonNull(metaStore, "metaStore"); - this.dataSourceExtractor - = Objects.requireNonNull(dataSourceExtractor, "dataSourceExtractor"); + this.metaStore = requireNonNull(metaStore, "metaStore"); + this.dataSourceExtractor = requireNonNull(dataSourceExtractor, "dataSourceExtractor"); } @Override diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/DataSourceExtractor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/DataSourceExtractor.java index 9a1dc7f85635..46a4518da836 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/DataSourceExtractor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/DataSourceExtractor.java @@ -44,9 +44,10 @@ class DataSourceExtractor { private SourceName rightAlias; private SourceName rightName; - private final Set commonFieldNames = new HashSet<>(); - private final Set leftFieldNames = new HashSet<>(); - private final Set rightFieldNames = new HashSet<>(); + private final Set allSources = new HashSet<>(); + private final Set commonColumnNames = new HashSet<>(); + private final Set leftColumnNames = new HashSet<>(); + private final Set rightColumnNames = new HashSet<>(); private boolean isJoin = false; @@ -56,7 +57,7 @@ class DataSourceExtractor { public void extractDataSources(final AstNode node) { new Visitor().process(node, null); - commonFieldNames.addAll(Sets.intersection(leftFieldNames, rightFieldNames)); + commonColumnNames.addAll(Sets.intersection(leftColumnNames, rightColumnNames)); } public SourceName getFromAlias() { @@ -71,16 +72,12 @@ public SourceName getRightAlias() { return rightAlias; } - public Set getCommonFieldNames() { - return Collections.unmodifiableSet(commonFieldNames); + public Set getAllSources() { + return Collections.unmodifiableSet(allSources); } - public Set getLeftFieldNames() { - return Collections.unmodifiableSet(leftFieldNames); - } - - public Set getRightFieldNames() { - return Collections.unmodifiableSet(rightFieldNames); + public Set getCommonColumnNames() { + return Collections.unmodifiableSet(commonColumnNames); } public SourceName getFromName() { @@ -101,15 +98,15 @@ public boolean isJoin() { public SourceName getAliasFor(final ColumnName columnName) { if (isJoin) { - if (commonFieldNames.contains(columnName)) { + if (commonColumnNames.contains(columnName)) { throw new KsqlException("Column '" + columnName.text() + "' is ambiguous."); } - if (leftFieldNames.contains(columnName)) { + if (leftColumnNames.contains(columnName)) { return leftAlias; } - if (rightFieldNames.contains(columnName)) { + if (rightColumnNames.contains(columnName)) { return rightAlias; } @@ -130,9 +127,12 @@ public Void visitRelation(final Relation relation, final Void ctx) { public Void visitAliasedRelation(final AliasedRelation relation, final Void ctx) { fromAlias = relation.getAlias(); fromName = ((Table) relation.getRelation()).getName(); - if (metaStore.getSource(fromName) == null) { + final DataSource source = metaStore.getSource(fromName); + if (source == null) { throw new KsqlException(fromName.text() + " does not exist."); } + + allSources.add(source); return null; } @@ -149,7 +149,7 @@ public Void visitJoin(final Join join, final Void ctx) { throw new KsqlException(((Table) left.getRelation()).getName().text() + " does not " + "exist."); } - addFieldNames(leftDataSource.getSchema(), leftFieldNames); + addFieldNames(leftDataSource.getSchema(), leftColumnNames); final AliasedRelation right = (AliasedRelation) join.getRight(); rightAlias = right.getAlias(); rightName = ((Table) right.getRelation()).getName(); @@ -160,7 +160,9 @@ public Void visitJoin(final Join join, final Void ctx) { throw new KsqlException(((Table) right.getRelation()).getName().text() + " does not " + "exist."); } - addFieldNames(rightDataSource.getSchema(), rightFieldNames); + addFieldNames(rightDataSource.getSchema(), rightColumnNames); + allSources.add(leftDataSource); + allSources.add(rightDataSource); return null; } } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/DataSourceExtractorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/DataSourceExtractorTest.java index 4e1e69b7e505..460be314d644 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/DataSourceExtractorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/DataSourceExtractorTest.java @@ -16,12 +16,15 @@ package io.confluent.ksql.engine.rewrite; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.MetaStore; +import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.AstBuilder; import io.confluent.ksql.parser.DefaultKsqlParser; @@ -30,6 +33,7 @@ import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.MetaStoreFixture; import java.util.List; +import java.util.stream.Collectors; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -183,6 +187,37 @@ public void shouldThrowIfRightJoinSourceDoesNotExist() { extractor.extractDataSources(stmt); } + @Test + public void shouldCaptureDataSources() { + // Given: + final AstNode stmt = givenQuery("SELECT * FROM TEST1;"); + + // When: + extractor.extractDataSources(stmt); + + // Then: + final List names = extractor.getAllSources().stream() + .map(DataSource::getName) + .collect(Collectors.toList()); + assertThat(names, contains(TEST1)); + } + + @Test + public void shouldCaptureJoinDataSources() { + // Given: + final AstNode stmt = givenQuery("SELECT * FROM TEST1 JOIN TEST2" + + " ON test1.col1 = test2.col1;"); + + // When: + extractor.extractDataSources(stmt); + + // Then: + final List names = extractor.getAllSources().stream() + .map(DataSource::getName) + .collect(Collectors.toList()); + assertThat(names, containsInAnyOrder(TEST1, TEST2)); + } + private static AstNode givenQuery(final String sql) { final List statements = new DefaultKsqlParser().parse(sql); assertThat(statements, hasSize(1)); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java index ff24f279a0dc..8af563cbfa29 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java @@ -529,7 +529,7 @@ public void shouldQueryMaterializedTableWithKeyFieldsInProjection() { final LogicalSchema schema = schema( "USERID", SqlTypes.STRING, - "KSQL_COL_1", SqlTypes.BIGINT, + "KSQL_COL_0", SqlTypes.BIGINT, "USERID_2", SqlTypes.STRING ); 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 f821ab36c34d..a3c3c9ccc930 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 @@ -205,8 +205,8 @@ public void shouldCreateExecutionPlan() { final String[] lines = planText.split("\n"); assertThat(lines[0], startsWith( - " > [ PROJECT ] | Schema: ROWKEY BIGINT KEY, COL0 BIGINT, KSQL_COL_1 DOUBLE, " - + "KSQL_COL_2 BIGINT |")); + " > [ PROJECT ] | Schema: ROWKEY BIGINT KEY, COL0 BIGINT, KSQL_COL_0 DOUBLE, " + + "KSQL_COL_1 BIGINT |")); assertThat(lines[1], startsWith( "\t\t > [ AGGREGATE ] | Schema: ROWKEY BIGINT KEY, KSQL_INTERNAL_COL_0 BIGINT, " + "KSQL_INTERNAL_COL_1 DOUBLE, KSQL_AGG_VARIABLE_0 DOUBLE, " diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java index 8386ef29a067..29d7c5925f99 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/AggregateNodeTest.java @@ -247,8 +247,8 @@ public void shouldBuildCorrectAggregateSchema() { // Then: assertThat(stream.getSchema().value(), contains( valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT), - valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.DOUBLE), - valueColumn(ColumnName.of("KSQL_COL_2"), SqlTypes.BIGINT))); + valueColumn(ColumnName.of("KSQL_COL_0"), SqlTypes.DOUBLE), + valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.BIGINT))); } @Test diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java index 3ad6e9675df5..581b890bc295 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKStreamTest.java @@ -421,8 +421,8 @@ public void testSelectWithExpression() { // Then: assertThat(projectedSchemaKStream.getSchema().value(), contains( valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT), - valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.INTEGER), - valueColumn(ColumnName.of("KSQL_COL_2"), SqlTypes.DOUBLE) + valueColumn(ColumnName.of("KSQL_COL_0"), SqlTypes.INTEGER), + valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.DOUBLE) )); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java index 897a32b02503..7ac4e1ead971 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java @@ -375,8 +375,8 @@ public void testSelectWithExpression() { // Then: assertThat(projectedSchemaKStream.getSchema().value(), contains( valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT), - valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.INTEGER), - valueColumn(ColumnName.of("KSQL_COL_2"), SqlTypes.DOUBLE) + valueColumn(ColumnName.of("KSQL_COL_0"), SqlTypes.INTEGER), + valueColumn(ColumnName.of("KSQL_COL_1"), SqlTypes.DOUBLE) )); } diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/plan.json new file mode 100644 index 000000000000..d907ce71c4cf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE DOUBLE) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` DOUBLE", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` DOUBLE" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/spec.json new file mode 100644 index 000000000000..81b32a7e7312 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/spec.json @@ -0,0 +1,36 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639495109, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0.0" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,0.0" + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,0.0" + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,1" + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,2" + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/count_-_count/6.0.0_1584639495109/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/plan.json new file mode 100644 index 000000000000..baa2f2684e8f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA DOUBLE) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` DOUBLE", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` DOUBLE KEY, `DATA` DOUBLE, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` DOUBLE" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/spec.json new file mode 100644 index 000000000000..160ae3c54a96 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498278, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : "0.1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "0.2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "0.1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "0.2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "0.1" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 0.1, + "value" : "0.1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 0.2, + "value" : "0.2,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 0.1, + "value" : "0.1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 0.2, + "value" : "0.2,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : 0.1, + "value" : "0.1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 0.1, + "value" : "0.1,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 0.2, + "value" : "0.2,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 0.1, + "value" : "0.1,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 0.2, + "value" : "0.2,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 0.1, + "value" : "0.1,0,3" + }, { + "topic" : "OUTPUT", + "key" : 0.1, + "value" : "0.1,1" + }, { + "topic" : "OUTPUT", + "key" : 0.2, + "value" : "0.2,1" + }, { + "topic" : "OUTPUT", + "key" : 0.1, + "value" : "0.1,2" + }, { + "topic" : "OUTPUT", + "key" : 0.2, + "value" : "0.2,2" + }, { + "topic" : "OUTPUT", + "key" : 0.1, + "value" : "0.1,3" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_double_field_with_re-key_(stream-_table)/6.0.0_1584639498278/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_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/plan.json new file mode 100644 index 000000000000..db0d8f7a8745 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(1) KSQL_COL_0,\n COUNT(*) KSQL_COL_1,\n TEST.DATA COPY\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT, `KSQL_COL_1` BIGINT, `COPY` STRING", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1", "1 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)", "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0", "KSQL_AGG_VARIABLE_1 AS KSQL_COL_1", "KSQL_INTERNAL_COL_0 AS COPY" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/spec.json new file mode 100644 index 000000000000..f615dedea9db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499317, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 3 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 1, + "KSQL_AGG_VARIABLE_0" : 1, + "KSQL_AGG_VARIABLE_1" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 2, + "KSQL_AGG_VARIABLE_0" : 1, + "KSQL_AGG_VARIABLE_1" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 3, + "KSQL_AGG_VARIABLE_0" : 2, + "KSQL_AGG_VARIABLE_1" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1, + "KSQL_COL_1" : 1, + "COPY" : "d1" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1, + "KSQL_COL_1" : 1, + "COPY" : "d2" + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2, + "KSQL_COL_1" : 2, + "COPY" : "d1" + }, + "timestamp" : 3 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_duplicate_fields_(stream-_table)/6.0.0_1584639499317/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_-_field_(stream-_table)/6.0.0_1584639497625/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/plan.json new file mode 100644 index 000000000000..b5a59c9d6534 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', KEY='data', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/spec.json new file mode 100644 index 000000000000..53d0e854636f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497625, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : "d1" + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : "d2" + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : "d1" + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : "d2" + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : "d1" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,3" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,1" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "d2,1" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,2" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "d2,2" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,3" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)/6.0.0_1584639497625/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/plan.json new file mode 100644 index 000000000000..51a216409743 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', KEY='data', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/spec.json new file mode 100644 index 000000000000..b8df74591334 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497651, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497651/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/plan.json new file mode 100644 index 000000000000..d6f3ba9ff0af --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', KEY='data', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/spec.json new file mode 100644 index 000000000000..7b5ded2a9696 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497676, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_JSON/6.0.0_1584639497676/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/plan.json new file mode 100644 index 000000000000..c2046769f75e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', KEY='data', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/spec.json new file mode 100644 index 000000000000..30cfb9a4de83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497701, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d2", + "value" : { + "DATA" : "d2" + }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "d1", + "value" : { + "DATA" : "d1" + }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497701/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/plan.json new file mode 100644 index 000000000000..e8d1ce2ec193 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/spec.json new file mode 100644 index 000000000000..a0c9249fa8bf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498255, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : "d1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d1" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : "d1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : "d2,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : "d1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : "d2,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : "d1,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : "d2,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : "d1,0,3" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,1" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "d2,1" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,2" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "d2,2" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,3" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)/6.0.0_1584639498255/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_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/plan.json new file mode 100644 index 000000000000..677c5c6fd596 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/spec.json new file mode 100644 index 000000000000..9919f36455cf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498304, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 3 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_AVRO/6.0.0_1584639498304/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_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/plan.json new file mode 100644 index 000000000000..17c16539398d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/spec.json new file mode 100644 index 000000000000..61371e89f6d4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498331, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 3 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_JSON/6.0.0_1584639498331/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_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/plan.json new file mode 100644 index 000000000000..95e7d1e61eb0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/spec.json new file mode 100644 index 000000000000..577077df0c92 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498357, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d2" + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "DATA" : "d1" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d2", + "value" : { + "KSQL_INTERNAL_COL_0" : "d2", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "d1", + "value" : { + "KSQL_INTERNAL_COL_0" : "d1", + "KSQL_INTERNAL_COL_1" : 0, + "KSQL_AGG_VARIABLE_0" : 3 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : { + "DATA" : "d2", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : { + "DATA" : "d1", + "KSQL_COL_0" : 3 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_field_with_re-key_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639498357/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/plan.json new file mode 100644 index 000000000000..d3cceb2539eb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/spec.json new file mode 100644 index 000000000000..9b9a1fe45920 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497779, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,a" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,2" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : "3,a,0,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,2" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,2" + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : "3,a,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)/6.0.0_1584639497779/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/plan.json new file mode 100644 index 000000000000..044c5c4d532c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/spec.json new file mode 100644 index 000000000000..1a411c2de8a8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497855, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "KSQL_INTERNAL_COL_0" : 3, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_AVRO/6.0.0_1584639497855/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/plan.json new file mode 100644 index 000000000000..84a290534a85 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/spec.json new file mode 100644 index 000000000000..6cb963776cbb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497894, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "KSQL_INTERNAL_COL_0" : 3, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_JSON/6.0.0_1584639497894/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/plan.json new file mode 100644 index 000000000000..557f3fc23f64 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/spec.json new file mode 100644 index 000000000000..5c74e0207830 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497941, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "F1" : 3, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 2 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|3", + "value" : { + "KSQL_INTERNAL_COL_0" : 3, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|3", + "value" : { + "F1" : 3, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(stream-_table)_-_format_-_PROTOBUF/6.0.0_1584639497941/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/plan.json new file mode 100644 index 000000000000..3aa1266c65cc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/spec.json new file mode 100644 index 000000000000..3a8ac48c6a47 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/spec.json @@ -0,0 +1,88 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498040, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,b" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a" + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,1" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : "2,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : "1,b,0,0" + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : "1,a,0,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,1" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "1,b,1" + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : "2,b,0" + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : "1,b,0" + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : "1,a,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)/6.0.0_1584639498040/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/plan.json new file mode 100644 index 000000000000..375518439a93 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/spec.json new file mode 100644 index 000000000000..617cadd1a3bb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/spec.json @@ -0,0 +1,163 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498084, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_AVRO/6.0.0_1584639498084/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/plan.json new file mode 100644 index 000000000000..b6bdd363356b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/spec.json new file mode 100644 index 000000000000..8b0a2771df76 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/spec.json @@ -0,0 +1,163 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498175, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_JSON/6.0.0_1584639498175/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/plan.json new file mode 100644 index 000000000000..536ce636eb98 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2, TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/spec.json new file mode 100644 index 000000000000..c09a7cb1cb29 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/spec.json @@ -0,0 +1,163 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498222, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "F1" : 2, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "b" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : null + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "F1" : 1, + "F2" : "a" + } + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|2", + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "b|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "b", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 0 + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "a|+|1", + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : "a", + "KSQL_INTERNAL_COL_2" : 0, + "KSQL_AGG_VARIABLE_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|2", + "value" : { + "F1" : 2, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "b|+|1", + "value" : { + "F1" : 1, + "F2" : "b", + "KSQL_COL_0" : 0 + } + }, { + "topic" : "OUTPUT", + "key" : "a|+|1", + "value" : { + "F1" : 1, + "F2" : "a", + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/topology new file mode 100644 index 000000000000..35fd727d4c26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_fields_(table-_table)_-_format_-_PROTOBUF/6.0.0_1584639498222/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/plan.json new file mode 100644 index 000000000000..5ee729fb3901 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.ID ID,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ID` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` INTEGER" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/spec.json new file mode 100644 index 000000000000..5bfa2b243deb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/spec.json @@ -0,0 +1,132 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497739, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "a", + "value" : { + "ID" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "b", + "value" : { + "ID" : 2 + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "c", + "value" : { + "ID" : 1 + }, + "timestamp" : 3 + }, { + "topic" : "test_topic", + "key" : "d", + "value" : { + "ID" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "test_topic", + "key" : "e", + "value" : { + "ID" : 1 + }, + "timestamp" : 5 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 1, + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : 1, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 2, + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : 2, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 1, + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : 3, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 2, + "value" : { + "KSQL_INTERNAL_COL_0" : 2, + "KSQL_INTERNAL_COL_1" : 4, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 1, + "value" : { + "KSQL_INTERNAL_COL_0" : 1, + "KSQL_INTERNAL_COL_1" : 5, + "KSQL_AGG_VARIABLE_0" : 3 + }, + "timestamp" : 5 + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ID" : 1, + "KSQL_COL_0" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "ID" : 2, + "KSQL_COL_0" : 1 + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ID" : 1, + "KSQL_COL_0" : 2 + }, + "timestamp" : 3 + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "ID" : 2, + "KSQL_COL_0" : 2 + }, + "timestamp" : 4 + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ID" : 1, + "KSQL_COL_0" : 3 + }, + "timestamp" : 5 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_int_field_(stream-_table)/6.0.0_1584639497739/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/plan.json new file mode 100644 index 000000000000..b3ad8d89a270 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (DATA STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.DATA DATA,\n (COUNT(*) * 2) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.DATA\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `DATA` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "DATA", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `DATA` STRING" + }, + "selectExpressions" : [ "DATA AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS DATA", "(KSQL_AGG_VARIABLE_0 * 2) AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/spec.json new file mode 100644 index 000000000000..2092f57199a1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/spec.json @@ -0,0 +1,36 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498401, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : "d1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "d1" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,2" + }, { + "topic" : "OUTPUT", + "key" : "d2", + "value" : "d2,2" + }, { + "topic" : "OUTPUT", + "key" : "d1", + "value" : "d1,4" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_(stream-_table)/6.0.0_1584639498401/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/plan.json new file mode 100644 index 000000000000..85d69db50c05 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ITEM INTEGER, COST INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ITEM` INTEGER, `COST` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.ITEM ITEM,\n TEST.COST COST,\n (TEST.COST * COUNT(*)) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.ITEM, TEST.COST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `ITEM` INTEGER, `COST` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ITEM` INTEGER, `COST` INTEGER" + }, + "selectExpressions" : [ "ITEM AS KSQL_INTERNAL_COL_0", "COST AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ITEM", "KSQL_INTERNAL_COL_1 AS COST", "(KSQL_INTERNAL_COL_1 * KSQL_AGG_VARIABLE_0) AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/spec.json new file mode 100644 index 000000000000..01f8f77bb578 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/spec.json @@ -0,0 +1,44 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639498449, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : "1,10" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "1,20" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "2,30" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "1,10" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1|+|10", + "value" : "1,10,10" + }, { + "topic" : "OUTPUT", + "key" : "1|+|20", + "value" : "1,20,20" + }, { + "topic" : "OUTPUT", + "key" : "2|+|30", + "value" : "2,30,30" + }, { + "topic" : "OUTPUT", + "key" : "1|+|10", + "value" : "1,10,20" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_aggregate_arithmetic_involving_source_field_(stream-_table)/6.0.0_1584639498449/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/plan.json new file mode 100644 index 000000000000..17e90cfa2027 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/plan.json @@ -0,0 +1,199 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F2 F2,\n SUM(TEST.F1) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2\nHAVING (TEST.F2 = 'test')\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F2` STRING, `KSQL_COL_0` INTEGER", + "keyField" : "F2", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F2 AS KSQL_INTERNAL_COL_0", "F1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "SUM(KSQL_INTERNAL_COL_1)" ] + }, + "filterExpression" : "(KSQL_INTERNAL_COL_0 = 'test')" + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/spec.json new file mode 100644 index 000000000000..88fc67b4b8fd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499173, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,test" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "3,test" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "b", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "test", + "value" : "test,2" + }, { + "topic" : "OUTPUT", + "key" : "b", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "test", + "value" : "test,5" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constant_having_(stream-table)/6.0.0_1584639499173/topology @@ -0,0 +1,49 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000014 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000014 (stores: []) + --> KSTREAM-SINK-0000000015 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000015 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000014 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/plan.json new file mode 100644 index 000000000000..d3611265f792 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n 'some constant' F3,\n COUNT(TEST.F1) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F3` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_0)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "'some constant' AS F3", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/spec.json new file mode 100644 index 000000000000..296848e08dfb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499222, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,some constant,1" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2,some constant,1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,some constant,2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2,some constant,2" + }, { + "topic" : "OUTPUT", + "key" : 3, + "value" : "3,some constant,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_constants_in_the_projection_(stream-_table)/6.0.0_1584639499222/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/plan.json new file mode 100644 index 000000000000..6ee5ae3a1c93 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER, F2 STRING, F3 INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING, `F3` INTEGER", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F3, (TEST.F2, TEST.F1)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "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", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `F2` STRING, `F3` INTEGER" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F2 AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2", "F3 AS KSQL_INTERNAL_COL_3" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_3", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2", "KSQL_INTERNAL_COL_3" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_INTERNAL_COL_1 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/spec.json new file mode 100644 index 000000000000..da95ef035a53 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639497987, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a,-1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b,-2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,a,-1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,b,-2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,a,-3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "-1|+|a|+|1", + "value" : "1,a,1" + }, { + "topic" : "OUTPUT", + "key" : "-2|+|b|+|2", + "value" : "2,b,1" + }, { + "topic" : "OUTPUT", + "key" : "-1|+|a|+|1", + "value" : "1,a,2" + }, { + "topic" : "OUTPUT", + "key" : "-2|+|b|+|2", + "value" : "2,b,2" + }, { + "topic" : "OUTPUT", + "key" : "-3|+|a|+|3", + "value" : "3,a,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_groupings_(stream-_table)/6.0.0_1584639497987/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/plan.json new file mode 100644 index 000000000000..66245270de56 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/plan.json @@ -0,0 +1,198 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY INTEGER KEY, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY='f1', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nHAVING (SUM(TEST.F1) > 1)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)", "SUM(KSQL_INTERNAL_COL_0)" ] + }, + "filterExpression" : "(KSQL_AGG_VARIABLE_1 > 1)" + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/spec.json new file mode 100644 index 000000000000..8092a0d7fe0e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499019, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2,1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2,2" + }, { + "topic" : "OUTPUT", + "key" : 3, + "value" : "3,1" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/topology new file mode 100644 index 000000000000..cb6f055df6f7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(stream-_table)/6.0.0_1584639499019/topology @@ -0,0 +1,34 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000009 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000009 (stores: []) + --> KSTREAM-SINK-0000000010 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000010 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000009 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/plan.json new file mode 100644 index 000000000000..564022ba055a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/plan.json @@ -0,0 +1,199 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (F0 INTEGER, F1 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `F0` INTEGER, `F1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n SUM(TEST.F0) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nHAVING (COUNT(TEST.F1) > 0)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `KSQL_COL_0` INTEGER", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `F0` INTEGER, `F1` INTEGER" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0", "F0 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "SUM(KSQL_INTERNAL_COL_1)", "COUNT(KSQL_INTERNAL_COL_0)" ] + }, + "filterExpression" : "(KSQL_AGG_VARIABLE_1 > 0)" + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/spec.json new file mode 100644 index 000000000000..e8f64bb6ff33 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/spec.json @@ -0,0 +1,56 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499047, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : "1,0" + }, { + "topic" : "test_topic", + "key" : "2", + "value" : "2,1" + }, { + "topic" : "test_topic", + "key" : "3", + "value" : "3,0" + }, { + "topic" : "test_topic", + "key" : "1", + "value" : null + }, { + "topic" : "test_topic", + "key" : "2", + "value" : "2,0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : "0,1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,2" + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : "0,4" + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : "0,3" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : "0,5" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/topology new file mode 100644 index 000000000000..d0e6a3373a3f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_(table-_table)/6.0.0_1584639499047/topology @@ -0,0 +1,49 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000014 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000014 (stores: []) + --> KSTREAM-SINK-0000000015 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000015 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000014 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/plan.json new file mode 100644 index 000000000000..2471af216951 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/plan.json @@ -0,0 +1,199 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F2 F2,\n COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F2\nHAVING (SUM(TEST.F1) > 10)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F2` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "F2", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F2 AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1", "F1 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)", "SUM(KSQL_INTERNAL_COL_2)" ] + }, + "filterExpression" : "(KSQL_AGG_VARIABLE_1 > 10)" + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F2", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/spec.json new file mode 100644 index 000000000000..079bf3eb1409 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/spec.json @@ -0,0 +1,60 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499125, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "-", + "value" : "5,a" + }, { + "topic" : "test_topic", + "key" : "-", + "value" : "10,b" + }, { + "topic" : "test_topic", + "key" : "-", + "value" : "6,a" + }, { + "topic" : "test_topic", + "key" : "-", + "value" : "1,b" + }, { + "topic" : "test_topic", + "key" : "-", + "value" : "-1,a" + }, { + "topic" : "test_topic", + "key" : "-", + "value" : "1,a" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "b", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "a", + "value" : "a,2" + }, { + "topic" : "OUTPUT", + "key" : "b", + "value" : "b,2" + }, { + "topic" : "OUTPUT", + "key" : "a", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "a", + "value" : "a,4" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_having_expression_on_non-group-by_field_(stream-_table)/6.0.0_1584639499125/topology @@ -0,0 +1,49 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000014 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000014 (stores: []) + --> KSTREAM-SINK-0000000015 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000015 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000014 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/plan.json new file mode 100644 index 000000000000..ca495e81c86a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/plan.json @@ -0,0 +1,199 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (F1 INTEGER, F2 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n TEST.F1 F1,\n COUNT(TEST.F1) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.F1\nHAVING ((COUNT(TEST.F1) > 1) AND (TEST.F1 = 1))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `F1` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "F1", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "Aggregate/HavingFilter" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `F1` INTEGER, `F2` STRING" + }, + "selectExpressions" : [ "F1 AS KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_0)", "COUNT(KSQL_INTERNAL_COL_0)" ] + }, + "filterExpression" : "((KSQL_AGG_VARIABLE_1 > 1) AND (KSQL_INTERNAL_COL_0 = 1))" + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS F1", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/spec.json new file mode 100644 index 000000000000..1fde907efb89 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/spec.json @@ -0,0 +1,60 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639499083, + "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" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : "1,a" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,b" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "1,test" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,test" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "2,test" + }, { + "topic" : "test_topic", + "key" : "0", + "value" : "1,test" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : null + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1,3" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_with_multiple_having_expressions_(stream-_table)/6.0.0_1584639499083/topology @@ -0,0 +1,49 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-FILTER-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000003 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> Aggregate-GroupBy-repartition-filter + <-- KSTREAM-FILTER-0000000003 + Processor: Aggregate-GroupBy-repartition-filter (stores: []) + --> Aggregate-GroupBy-repartition-sink + <-- Aggregate-GroupBy + Sink: Aggregate-GroupBy-repartition-sink (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy-repartition-filter + + Sub-topology: 1 + Source: Aggregate-GroupBy-repartition-source (topics: [Aggregate-GroupBy-repartition]) + --> KSTREAM-AGGREGATE-0000000005 + Processor: KSTREAM-AGGREGATE-0000000005 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-GroupBy-repartition-source + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-HavingFilter-ApplyPredicate + <-- KSTREAM-AGGREGATE-0000000005 + Processor: Aggregate-HavingFilter-ApplyPredicate (stores: []) + --> Aggregate-HavingFilter-Filter + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-HavingFilter-Filter (stores: []) + --> Aggregate-HavingFilter-PostProcess + <-- Aggregate-HavingFilter-ApplyPredicate + Processor: Aggregate-HavingFilter-PostProcess (stores: []) + --> Aggregate-Project + <-- Aggregate-HavingFilter-Filter + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000014 + <-- Aggregate-HavingFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000014 (stores: []) + --> KSTREAM-SINK-0000000015 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000015 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000014 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/plan.json new file mode 100644 index 000000000000..5ebc902a1e47 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "MAX(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/spec.json new file mode 100644 index 000000000000..c4471cb3765a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/spec.json @@ -0,0 +1,227 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639500009, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,100", + "timestamp" : 30000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,6", + "timestamp" : 45000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,300", + "timestamp" : 50000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,100", + "timestamp" : 35000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,2000", + "timestamp" : 40000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,5", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,5", + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 45000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 45000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,6", + "timestamp" : 45000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,300", + "timestamp" : 50000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,300", + "timestamp" : 50000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,300", + "timestamp" : 50000, + "window" : { + "start" : 50000, + "end" : 80000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,100", + "timestamp" : 35000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,100", + "timestamp" : 35000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,100", + "timestamp" : 35000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,2000", + "timestamp" : 40000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,2000", + "timestamp" : 40000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,2000", + "timestamp" : 40000, + "window" : { + "start" : 40000, + "end" : 70000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_max_hopping/6.0.0_1584639500009/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/plan.json new file mode 100644 index 000000000000..aaf44dbfa41f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MIN(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "MIN(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " HOPPING ( SIZE 30 SECONDS , ADVANCE BY 10 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/spec.json new file mode 100644 index 000000000000..e145c1810482 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/spec.json @@ -0,0 +1,87 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639500036, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,100", + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,5", + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 20000, + "end" : 50000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/hopping-windows_-_min_hopping/6.0.0_1584639500036/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/plan.json new file mode 100644 index 000000000000..c1236509e4ef --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/spec.json new file mode 100644 index 000000000000..54ede6efec9a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509270, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ALIASED" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/topology new file mode 100644 index 000000000000..5144c38f80a3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509270/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_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/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/plan.json new file mode 100644 index 000000000000..1c909342b17d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/spec.json new file mode 100644 index 000000000000..85c29019b15c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509245, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : "", + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "FOO" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/topology new file mode 100644 index 000000000000..5144c38f80a3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509245/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_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/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/plan.json new file mode 100644 index 000000000000..ee7432f854c8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.BAR ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.BAR\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/spec.json new file mode 100644 index 000000000000..2580b03e3068 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509492, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "ALIASED" : 2, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/topology new file mode 100644 index 000000000000..5144c38f80a3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509492/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_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/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/plan.json new file mode 100644 index 000000000000..f58993c97b95 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.BAR BAR,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.BAR\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `BAR` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "BAR", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS BAR", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/spec.json new file mode 100644 index 000000000000..08366812fa5f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509462, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "BAR" : 2, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/topology new file mode 100644 index 000000000000..5144c38f80a3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509462/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_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/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/plan.json new file mode 100644 index 000000000000..81686f9ba2c6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/spec.json new file mode 100644 index 000000000000..ee491e91a261 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509422, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ALIASED" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/topology new file mode 100644 index 000000000000..e8b91bfb01d7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509422/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/plan.json new file mode 100644 index 000000000000..54461bbc3b1f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/spec.json new file mode 100644 index 000000000000..b19d097b408a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509406, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "FOO" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/topology new file mode 100644 index 000000000000..e8b91bfb01d7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509406/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/plan.json new file mode 100644 index 000000000000..8c453391233d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO FOO,\n INPUT.BAR BAR,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.BAR, INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `FOO` INTEGER, `BAR` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "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" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "BAR AS KSQL_INTERNAL_COL_1", "ROWTIME AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1", "KSQL_INTERNAL_COL_2" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_2)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS FOO", "KSQL_INTERNAL_COL_1 AS BAR", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/spec.json new file mode 100644 index 000000000000..8ae864cc350f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/spec.json @@ -0,0 +1,27 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509846, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "2|+|1", + "value" : { + "FOO" : 1, + "BAR" : 2, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/topology new file mode 100644 index 000000000000..5144c38f80a3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_stream___initially_set___group_by_multiple___key_in_value___no_aliasing/6.0.0_1584639509846/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_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/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/plan.json new file mode 100644 index 000000000000..4320a566821e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (FOO INTEGER) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `FOO` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `FOO` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/spec.json new file mode 100644 index 000000000000..78d37d3fbbe8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/spec.json @@ -0,0 +1,25 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509596, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : "x", + "value" : { + "foo" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ALIASED" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___aliasing/6.0.0_1584639509596/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/plan.json new file mode 100644 index 000000000000..ff1493b97763 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (FOO INTEGER) WITH (KAFKA_TOPIC='input_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `FOO` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `FOO` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/spec.json new file mode 100644 index 000000000000..0faf63a96e7e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/spec.json @@ -0,0 +1,25 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509569, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : "x", + "value" : { + "foo" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "FOO" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_null___group_by_(-)___key_in_value___no_aliasing/6.0.0_1584639509569/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/plan.json new file mode 100644 index 000000000000..b746940daeb6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.BAR ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.BAR\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/spec.json new file mode 100644 index 000000000000..cac72c77f963 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509744, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "ALIASED" : 2, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___aliasing/6.0.0_1584639509744/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/plan.json new file mode 100644 index 000000000000..320c6dc352be --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.BAR BAR,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.BAR\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `BAR` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "BAR", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "BAR AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS BAR", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/spec.json new file mode 100644 index 000000000000..b3b49e416cb1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509721, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "BAR" : 2, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(different)___key_in_value___no_aliasing/6.0.0_1584639509721/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/plan.json new file mode 100644 index 000000000000..104610b865e2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO ALIASED,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `ALIASED` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "ALIASED", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ALIASED", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/spec.json new file mode 100644 index 000000000000..c5cf144b0e69 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509684, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "ALIASED" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___aliasing/6.0.0_1584639509684/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/plan.json new file mode 100644 index 000000000000..f622781e5d2e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT (ROWKEY INTEGER KEY, FOO INTEGER, BAR INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY='foo', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.FOO FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INPUT.FOO\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `KSQL_COL_0` BIGINT", + "keyField" : "FOO", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` INTEGER KEY, `FOO` INTEGER, `BAR` INTEGER" + }, + "selectExpressions" : [ "FOO AS KSQL_INTERNAL_COL_0", "ROWTIME AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/spec.json new file mode 100644 index 000000000000..53c664813782 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639509663, + "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" + }, + "inputs" : [ { + "topic" : "input_topic", + "key" : 1, + "value" : { + "foo" : 1, + "bar" : 2 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "FOO" : 1, + "KSQL_COL_0" : 1 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/topology new file mode 100644 index 000000000000..17651552af83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-field_-_table___initially_set___group_by_(same)___key_in_value___no_aliasing/6.0.0_1584639509663/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/plan.json new file mode 100644 index 000000000000..5c4ed08096c3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ROWKEY BIGINT KEY, VALUE BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` BIGINT KEY, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n EXP(INPUT.ROWKEY) EXP,\n COUNT(1) KSQL_COL_0\nFROM INPUT INPUT\nWINDOW TUMBLING ( SIZE 30 SECONDS ) \nGROUP BY INPUT.ROWKEY\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` BIGINT KEY, `EXP` DOUBLE, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ROWKEY AS KSQL_INTERNAL_COL_0", "1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " TUMBLING ( SIZE 30 SECONDS ) " + }, + "selectExpressions" : [ "EXP(KSQL_INTERNAL_COL_0) AS EXP", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/spec.json new file mode 100644 index 000000000000..38aef7612e08 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/spec.json @@ -0,0 +1,30 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639510231, + "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" + }, + "inputs" : [ { + "topic" : "input", + "key" : 10, + "value" : { + "value" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "EXP" : 22026.465794806718, + "KSQL_COL_0" : 1 + }, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/topology new file mode 100644 index 000000000000..4a7de4106512 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/key-schemas_-_windowed_table_explicit_non-STRING_ROWKEY_udf/6.0.0_1584639510231/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/plan.json new file mode 100644 index 000000000000..cb4fa519dde7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/plan.json @@ -0,0 +1,148 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n true KSQL_COL_0,\n true KSQL_COL_1,\n true KSQL_COL_2,\n false KSQL_COL_3,\n false KSQL_COL_4,\n false KSQL_COL_5\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `KSQL_COL_0` BOOLEAN, `KSQL_COL_1` BOOLEAN, `KSQL_COL_2` BOOLEAN, `KSQL_COL_3` BOOLEAN, `KSQL_COL_4` BOOLEAN, `KSQL_COL_5` BOOLEAN", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "true AS KSQL_COL_0", "true AS KSQL_COL_1", "true AS KSQL_COL_2", "false AS KSQL_COL_3", "false AS KSQL_COL_4", "false AS KSQL_COL_5" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/spec.json new file mode 100644 index 000000000000..4f00f87bea45 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/spec.json @@ -0,0 +1,18 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639510378, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : "0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : "0,true,true,true,false,false,false" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_BOOLEAN_literal/6.0.0_1584639510378/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/literals_-_DECIMAL_literal/6.0.0_1584639510528/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/plan.json new file mode 100644 index 000000000000..ce796b976bde --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/plan.json @@ -0,0 +1,148 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n 2.345 KSQL_COL_0\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `KSQL_COL_0` DECIMAL(4, 3)", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "2.345 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/spec.json new file mode 100644 index 000000000000..2c5d5331ee32 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/spec.json @@ -0,0 +1,18 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639510528, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : "", + "value" : "0" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : "0,2.345" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/literals_-_DECIMAL_literal/6.0.0_1584639510528/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/session-windows_-_max_session/6.0.0_1584639513840/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/plan.json new file mode 100644 index 000000000000..15e45b5446f0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW SESSION ( 30 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "MAX(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " SESSION ( 30 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/spec.json new file mode 100644 index 000000000000..d00641913feb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639513840, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,100,100", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,100,200", + "timestamp" : 40000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 0, + "type" : "SESSION" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : null, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 0, + "type" : "SESSION" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,5", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 10000, + "type" : "SESSION" + } + }, { + "topic" : "S2", + "key" : 1, + "value" : "1,100", + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 10000, + "type" : "SESSION" + } + }, { + "topic" : "S2", + "key" : 1, + "value" : null, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 10000, + "type" : "SESSION" + } + }, { + "topic" : "S2", + "key" : 1, + "value" : "1,200", + "timestamp" : 40000, + "window" : { + "start" : 10000, + "end" : 40000, + "type" : "SESSION" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/session-windows_-_max_session/6.0.0_1584639513840/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/plan.json new file mode 100644 index 000000000000..dbdcad479880 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/plan.json @@ -0,0 +1,148 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM ORDERS (ORDERTIME BIGINT, ORDERID BIGINT, ITEMID STRING, ORDERUNITS DOUBLE, ARRAYCOL ARRAY, MAPCOL MAP, ADDRESS STRUCT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "ORDERS", + "schema" : "`ROWKEY` STRING KEY, `ORDERTIME` BIGINT, `ORDERID` BIGINT, `ITEMID` STRING, `ORDERUNITS` DOUBLE, `ARRAYCOL` ARRAY, `MAPCOL` MAP, `ADDRESS` STRUCT<`NUMBER` BIGINT, `STREET` STRING, `CITY` STRING, `STATE` STRING, `ZIPCODE` BIGINT>", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 AS SELECT\n ORDERS.ITEMID ITEMID,\n EXTRACTJSONFIELD(ORDERS.ITEMID, '$.ITEMID') KSQL_COL_0\nFROM ORDERS ORDERS\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` STRING KEY, `ITEMID` STRING, `KSQL_COL_0` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "ORDERS" ], + "sink" : "S1", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ORDERTIME` BIGINT, `ORDERID` BIGINT, `ITEMID` STRING, `ORDERUNITS` DOUBLE, `ARRAYCOL` ARRAY, `MAPCOL` MAP, `ADDRESS` STRUCT<`NUMBER` BIGINT, `STREET` STRING, `CITY` STRING, `STATE` STRING, `ZIPCODE` BIGINT>" + }, + "selectExpressions" : [ "ITEMID AS ITEMID", "EXTRACTJSONFIELD(ITEMID, '$.ITEMID') AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1", + "timestampColumn" : null + }, + "queryId" : "CSAS_S1_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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/spec.json new file mode 100644 index 000000000000..03eca054a2dd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/spec.json @@ -0,0 +1,200 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639513982, + "schemas" : { + "CSAS_S1_0.KsqlTopic.Source" : "STRUCT, MAPCOL MAP, ADDRESS STRUCT> NOT NULL", + "CSAS_S1_0.S1" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : { + "ORDERID" : 1, + "ITEMID" : { + "ITEMID" : 6, + "CATEGORY" : { + "ID" : 2, + "NAME" : "Food" + }, + "NAME" : "Item_6" + }, + "ORDERTIME" : 1528224275715, + "MAPCOL" : { + "key3" : 3.8688222734507915, + "key2" : 5.878674158377773, + "key1" : 2.706938954083115 + }, + "ORDERUNITS" : 2.0, + "ADDRESS" : { + "CITY" : "CITY_0", + "STATE" : "STATE_1", + "STREET" : "STREET_4", + "NUMBER" : 376, + "ZIPCODE" : 621 + }, + "ARRAYCOL" : [ 6.27276558443913, 8.720822816653817, 4.904955205015469, 0.28466518164817933, 5.276269704236784 ] + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "100", + "value" : { + "ORDERID" : 2, + "ITEMID" : { + "ITEMID" : 6, + "CATEGORY" : { + "ID" : 2, + "NAME" : "Produce" + }, + "NAME" : "Item_6" + }, + "ORDERTIME" : 1528224280668, + "MAPCOL" : { + "key3" : 0.08025583241041634, + "key2" : 7.886688738692968, + "key1" : 7.997268326700826 + }, + "ORDERUNITS" : 4.0, + "ADDRESS" : { + "CITY" : "CITY_3", + "STATE" : "STATE_6", + "STREET" : "STREET_5", + "NUMBER" : 29, + "ZIPCODE" : 46 + }, + "ARRAYCOL" : [ 5.028181423106411, 4.223556791057725, 7.503771637501132, 1.8346470572995977, 8.628168574256188 ] + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "101", + "value" : { + "ORDERID" : 3, + "ITEMID" : { + "ITEMID" : 6, + "CATEGORY" : { + "ID" : 2, + "NAME" : "Produce" + }, + "NAME" : "Item_6" + }, + "ORDERTIME" : 1528224281566, + "MAPCOL" : { + "key3" : 8.232202829012866, + "key2" : 4.76749034853443, + "key1" : 3.908548556676262 + }, + "ORDERUNITS" : 6.0, + "ADDRESS" : { + "CITY" : "CITY_9", + "STATE" : "STATE_9", + "STREET" : "STREET_3", + "NUMBER" : 219, + "ZIPCODE" : 287 + }, + "ARRAYCOL" : [ 9.404053021473551, 2.005832055529364, 0.16252060679229574, 8.030440873506674, 2.6822009490877683 ] + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "101", + "value" : { + "ORDERID" : 4, + "ITEMID" : { + "ITEMID" : 2, + "CATEGORY" : { + "ID" : 1, + "NAME" : "Food" + }, + "NAME" : "Item_2" + }, + "ORDERTIME" : 1528224285603, + "MAPCOL" : { + "key3" : 8.58507015716884, + "key2" : 8.690191464522353, + "key1" : 3.966253991851106 + }, + "ORDERUNITS" : 3.0, + "ADDRESS" : { + "CITY" : "CITY_3", + "STATE" : "STATE_5", + "STREET" : "STREET_8", + "NUMBER" : 380, + "ZIPCODE" : 866 + }, + "ARRAYCOL" : [ 9.887072401304447, 5.217021497196517, 5.604857288119519, 4.628527278923561, 6.367135367927281 ] + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "101", + "value" : { + "ORDERID" : 5, + "ITEMID" : { + "ITEMID" : 5, + "CATEGORY" : { + "ID" : 1, + "NAME" : "Produce" + }, + "NAME" : "Item_5" + }, + "ORDERTIME" : 1528224286568, + "MAPCOL" : { + "key3" : 4.9160189684727, + "key2" : 1.3876747586974092, + "key1" : 6.688854425726891 + }, + "ORDERUNITS" : 5.0, + "ADDRESS" : { + "CITY" : "CITY_6", + "STATE" : "STATE_3", + "STREET" : "STREET_8", + "NUMBER" : 294, + "ZIPCODE" : 724 + }, + "ARRAYCOL" : [ 1.6856668854084866, 3.4970511301361484, 9.143163282671962, 2.196065628133206, 4.343961390870502 ] + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1", + "key" : "0", + "value" : { + "ITEMID" : "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Food\"},\"NAME\":\"Item_6\"}", + "KSQL_COL_0" : "6" + }, + "timestamp" : 0 + }, { + "topic" : "S1", + "key" : "100", + "value" : { + "ITEMID" : "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Produce\"},\"NAME\":\"Item_6\"}", + "KSQL_COL_0" : "6" + }, + "timestamp" : 0 + }, { + "topic" : "S1", + "key" : "101", + "value" : { + "ITEMID" : "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Produce\"},\"NAME\":\"Item_6\"}", + "KSQL_COL_0" : "6" + }, + "timestamp" : 0 + }, { + "topic" : "S1", + "key" : "101", + "value" : { + "ITEMID" : "{\"ITEMID\":2,\"CATEGORY\":{\"ID\":1,\"NAME\":\"Food\"},\"NAME\":\"Item_2\"}", + "KSQL_COL_0" : "2" + }, + "timestamp" : 0 + }, { + "topic" : "S1", + "key" : "101", + "value" : { + "ITEMID" : "{\"ITEMID\":5,\"CATEGORY\":{\"ID\":1,\"NAME\":\"Produce\"},\"NAME\":\"Item_5\"}", + "KSQL_COL_0" : "5" + }, + "timestamp" : 0 + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/topology new file mode 100644 index 000000000000..08dc66d3de05 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/simple-struct_-_simple_struct_read_struct_as_json_string_-_JSON/6.0.0_1584639513982/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: S1) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/plan.json new file mode 100644 index 000000000000..fa7623def4f8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/plan.json @@ -0,0 +1,148 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (TEXT STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `TEXT` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.TEXT TEXT,\n (INPUT.TEXT < 'b2') KSQL_COL_0\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `TEXT` STRING, `KSQL_COL_0` BOOLEAN", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `TEXT` STRING" + }, + "selectExpressions" : [ "TEXT AS TEXT", "(TEXT < 'b2') AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/spec.json new file mode 100644 index 000000000000..981e2c92989f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/spec.json @@ -0,0 +1,66 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639514494, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : "a1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "b1" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "B2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "b2" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "b3" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "b10" + }, { + "topic" : "test_topic", + "key" : "", + "value" : "b01" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : "a1,true" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "b1,true" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "B2,true" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "b2,false" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "b3,false" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "b10,true" + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : "b01,true" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/string_-___operator/6.0.0_1584639514494/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: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/plan.json similarity index 96% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/plan.json rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/plan.json index a9d8556430fc..357c57e5888a 100644 --- a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/plan.json +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/plan.json @@ -25,11 +25,11 @@ "queryPlan" : null }, { "@type" : "ksqlPlanV1", - "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.SHOULDTHROW SHOULDTHROW,\n TEST_UDTF(BAD_UDF(TEST.SHOULDTHROW)) KSQL_COL_1\nFROM TEST TEST\nEMIT CHANGES", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.SHOULDTHROW SHOULDTHROW,\n TEST_UDTF(BAD_UDF(TEST.SHOULDTHROW)) KSQL_COL_0\nFROM TEST TEST\nEMIT CHANGES", "ddlCommand" : { "@type" : "createStreamV1", "sourceName" : "OUTPUT", - "schema" : "`ROWKEY` STRING KEY, `SHOULDTHROW` BOOLEAN, `KSQL_COL_1` INTEGER", + "schema" : "`ROWKEY` STRING KEY, `SHOULDTHROW` BOOLEAN, `KSQL_COL_0` INTEGER", "keyField" : null, "timestampColumn" : null, "topicName" : "OUTPUT", @@ -86,7 +86,7 @@ }, "tableFunctions" : [ "TEST_UDTF(BAD_UDF(SHOULDTHROW))" ] }, - "selectExpressions" : [ "SHOULDTHROW AS SHOULDTHROW", "KSQL_SYNTH_0 AS KSQL_COL_1" ] + "selectExpressions" : [ "SHOULDTHROW AS SHOULDTHROW", "KSQL_SYNTH_0 AS KSQL_COL_0" ] }, "formats" : { "keyFormat" : { @@ -99,8 +99,7 @@ }, "options" : [ ] }, - "topicName" : "OUTPUT", - "timestampColumn" : null + "topicName" : "OUTPUT" }, "queryId" : "CSAS_OUTPUT_0" } @@ -121,7 +120,7 @@ "ksql.service.id" : "some.ksql.service.id", "ksql.internal.topic.min.insync.replicas" : "1", "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent5381695383456269897", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent828390110039021725", "ksql.internal.topic.replicas" : "1", "ksql.insert.into.values.enabled" : "true", "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/spec.json similarity index 88% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/spec.json rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/spec.json index 6d21d413704f..665cca2eb91f 100644 --- a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/spec.json +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/spec.json @@ -1,9 +1,9 @@ { "version" : "6.0.0", - "timestamp" : 1584715538172, + "timestamp" : 1585050801382, "schemas" : { "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", - "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" }, "inputs" : [ { "topic" : "test_topic", @@ -29,14 +29,14 @@ "key" : "a", "value" : { "SHOULDTHROW" : false, - "KSQL_COL_1" : 0 + "KSQL_COL_0" : 0 } }, { "topic" : "OUTPUT", "key" : "c", "value" : { "SHOULDTHROW" : false, - "KSQL_COL_1" : 0 + "KSQL_COL_0" : 0 } } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/topology similarity index 100% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1584715538172/topology rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTF_parameter_evaluation_that_throws/6.0.0_1585050801382/topology diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/plan.json similarity index 96% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/plan.json rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/plan.json index 0db26491d76d..e6ad0fedcc56 100644 --- a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/plan.json +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/plan.json @@ -25,11 +25,11 @@ "queryPlan" : null }, { "@type" : "ksqlPlanV1", - "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.SHOULDTHROW SHOULDTHROW,\n THROWING_UDTF(TEST.SHOULDTHROW) KSQL_COL_1\nFROM TEST TEST\nEMIT CHANGES", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.SHOULDTHROW SHOULDTHROW,\n THROWING_UDTF(TEST.SHOULDTHROW) KSQL_COL_0\nFROM TEST TEST\nEMIT CHANGES", "ddlCommand" : { "@type" : "createStreamV1", "sourceName" : "OUTPUT", - "schema" : "`ROWKEY` STRING KEY, `SHOULDTHROW` BOOLEAN, `KSQL_COL_1` BOOLEAN", + "schema" : "`ROWKEY` STRING KEY, `SHOULDTHROW` BOOLEAN, `KSQL_COL_0` BOOLEAN", "keyField" : null, "timestampColumn" : null, "topicName" : "OUTPUT", @@ -86,7 +86,7 @@ }, "tableFunctions" : [ "THROWING_UDTF(SHOULDTHROW)" ] }, - "selectExpressions" : [ "SHOULDTHROW AS SHOULDTHROW", "KSQL_SYNTH_0 AS KSQL_COL_1" ] + "selectExpressions" : [ "SHOULDTHROW AS SHOULDTHROW", "KSQL_SYNTH_0 AS KSQL_COL_0" ] }, "formats" : { "keyFormat" : { @@ -99,8 +99,7 @@ }, "options" : [ ] }, - "topicName" : "OUTPUT", - "timestampColumn" : null + "topicName" : "OUTPUT" }, "queryId" : "CSAS_OUTPUT_0" } @@ -121,7 +120,7 @@ "ksql.service.id" : "some.ksql.service.id", "ksql.internal.topic.min.insync.replicas" : "1", "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent5381695383456269897", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent828390110039021725", "ksql.internal.topic.replicas" : "1", "ksql.insert.into.values.enabled" : "true", "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/spec.json similarity index 86% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/spec.json rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/spec.json index e0229dc3efec..abd8c107730b 100644 --- a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/spec.json +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/spec.json @@ -1,9 +1,9 @@ { "version" : "6.0.0", - "timestamp" : 1584715538157, + "timestamp" : 1585050801320, "schemas" : { "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", - "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" }, "inputs" : [ { "topic" : "test_topic", @@ -29,14 +29,14 @@ "key" : "a", "value" : { "SHOULDTHROW" : false, - "KSQL_COL_1" : false + "KSQL_COL_0" : false } }, { "topic" : "OUTPUT", "key" : "c", "value" : { "SHOULDTHROW" : false, - "KSQL_COL_1" : false + "KSQL_COL_0" : false } } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/topology similarity index 100% rename from ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1584715538157/topology rename to ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_should_handle_UDTFs_that_throw/6.0.0_1585050801320/topology diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/plan.json new file mode 100644 index 000000000000..290c8f752d26 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/plan.json @@ -0,0 +1,155 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (F0 INTEGER, F1 INTEGER, F2 INTEGER, F3 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `F0` INTEGER, `F1` INTEGER, `F2` INTEGER, `F3` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.F0 F0,\n EXPLODE(ARRAY[ABS((TEST.F1 + TEST.F2)), ABS((TEST.F2 + TEST.F3)), ABS((TEST.F3 + TEST.F1))]) KSQL_COL_0\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `F0` INTEGER, `KSQL_COL_0` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamFlatMapV1", + "properties" : { + "queryContext" : "FlatMap" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `F0` INTEGER, `F1` INTEGER, `F2` INTEGER, `F3` INTEGER" + }, + "tableFunctions" : [ "EXPLODE(ARRAY[ABS((F1 + F2)), ABS((F2 + F3)), ABS((F3 + F1))])" ] + }, + "selectExpressions" : [ "F0 AS F0", "KSQL_SYNTH_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/spec.json new file mode 100644 index 000000000000..3dc37a27aeef --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/spec.json @@ -0,0 +1,41 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639515097, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : { + "ID" : 0, + "F0" : 1, + "F1" : 10, + "F2" : 11, + "F3" : 12 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "0", + "value" : { + "F0" : 1, + "KSQL_COL_0" : 21 + } + }, { + "topic" : "OUTPUT", + "key" : "0", + "value" : { + "F0" : 1, + "KSQL_COL_0" : 23 + } + }, { + "topic" : "OUTPUT", + "key" : "0", + "value" : { + "F0" : 1, + "KSQL_COL_0" : 22 + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/topology new file mode 100644 index 000000000000..a266798d8e3b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/table-functions_-_table_functions_with_complex_expressions/6.0.0_1584639515097/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> FlatMap + <-- KSTREAM-SOURCE-0000000000 + Processor: FlatMap (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- FlatMap + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/plan.json new file mode 100644 index 000000000000..7c84e9fbab7f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/plan.json @@ -0,0 +1,191 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TEST_UDAF(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "TEST_UDAF(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/spec.json new file mode 100644 index 000000000000..17604dcad0d5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/spec.json @@ -0,0 +1,68 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639515406, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,-2147483647" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5" + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,100" + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,6" + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,300" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,2000" + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,100" + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,-2147483647" + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,-2147483642" + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100" + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,106" + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,406" + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,-2147481642" + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,-2147481542" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_group_by/6.0.0_1584639515406/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/plan.json new file mode 100644 index 000000000000..dafc5cb0a91a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/plan.json @@ -0,0 +1,192 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, REGION STRING) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE SUM_ID_BY_REGION AS SELECT\n TEST.REGION REGION,\n TEST_UDAF(TEST.ID) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.REGION\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "SUM_ID_BY_REGION", + "schema" : "`ROWKEY` STRING KEY, `REGION` STRING, `KSQL_COL_0` BIGINT", + "keyField" : "REGION", + "timestampColumn" : null, + "topicName" : "SUM_ID_BY_REGION", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "SUM_ID_BY_REGION", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "SUM_ID_BY_REGION" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING" + }, + "selectExpressions" : [ "REGION AS KSQL_INTERNAL_COL_0", "ID AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "groupByExpressions" : [ "KSQL_INTERNAL_COL_0" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "TEST_UDAF(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS REGION", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "SUM_ID_BY_REGION", + "timestampColumn" : null + }, + "queryId" : "CTAS_SUM_ID_BY_REGION_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.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/spec.json new file mode 100644 index 000000000000..76cacc783f83 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/spec.json @@ -0,0 +1,64 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639515426, + "schemas" : { + "CTAS_SUM_ID_BY_REGION_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_SUM_ID_BY_REGION_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_SUM_ID_BY_REGION_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_SUM_ID_BY_REGION_0.SUM_ID_BY_REGION" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,alice,east" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,bob,east" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2,carol,west" + }, { + "topic" : "test_topic", + "key" : 3, + "value" : "3,dave,west" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "1,bob,west" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : null + } ], + "outputs" : [ { + "topic" : "SUM_ID_BY_REGION", + "key" : "east", + "value" : "east,0" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "east", + "value" : "east,1" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "west", + "value" : "west,2" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "west", + "value" : "west,5" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "east", + "value" : "east,0" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "west", + "value" : "west,6" + }, { + "topic" : "SUM_ID_BY_REGION", + "key" : "west", + "value" : "west,5" + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/topology new file mode 100644 index 000000000000..90b879cadb77 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/test-custom-udaf_-_test_udaf_on_a_table/6.0.0_1584639515426/topology @@ -0,0 +1,40 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-SOURCE-0000000001 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-FILTER-0000000004 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000006 + <-- KTABLE-FILTER-0000000004 + Sink: KSTREAM-SINK-0000000006 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000007 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000008 + Processor: KTABLE-AGGREGATE-0000000008 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000007 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000008 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000011 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000011 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000012 (topic: SUM_ID_BY_REGION) + <-- KTABLE-TOSTREAM-0000000011 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/plan.json new file mode 100644 index 000000000000..72155aee267b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW TUMBLING ( SIZE 30 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "MAX(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " TUMBLING ( SIZE 30 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/spec.json new file mode 100644 index 000000000000..c434a73b8d69 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/spec.json @@ -0,0 +1,117 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639515977, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,100", + "timestamp" : 30000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,6", + "timestamp" : 45000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,300", + "timestamp" : 50000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,100", + "timestamp" : 35000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,2000", + "timestamp" : 40000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,5", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 45000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,300", + "timestamp" : 50000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,100", + "timestamp" : 35000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,2000", + "timestamp" : 40000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_max_tumbling/6.0.0_1584639515977/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/plan.json new file mode 100644 index 000000000000..79800fb9d0fc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MIN(TEST.VALUE) KSQL_COL_0\nFROM TEST TEST\nWINDOW TUMBLING ( SIZE 30 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "VALUE AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "MIN(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " TUMBLING ( SIZE 30 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/spec.json new file mode 100644 index 000000000000..6b23c7a58632 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/spec.json @@ -0,0 +1,87 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639515994, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : "0,zero,0", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : "0,100,5", + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,100", + "timestamp" : 30000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,6", + "timestamp" : 45000 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100,100,300", + "timestamp" : 50000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : "0,0", + "timestamp" : 10000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,100", + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,6", + "timestamp" : 45000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,6", + "timestamp" : 50000, + "window" : { + "start" : 30000, + "end" : 60000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/tumbling-windows_-_min_tumbling/6.0.0_1584639515994/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/plan.json new file mode 100644 index 000000000000..d9b6b83d78a5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/plan.json @@ -0,0 +1,195 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY='ID', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n (TEST.WINDOWSTART / 2) KSQL_COL_0,\n (TEST.WINDOWEND / TEST.ID) KSQL_COL_1,\n COUNT(1) COUNT\nFROM TEST TEST\nWINDOW TUMBLING ( SIZE 30 SECONDS ) \nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `KSQL_COL_0` BIGINT, `KSQL_COL_1` BIGINT, `COUNT` BIGINT", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 30.000000000 + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamWindowedAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ], + "windowExpression" : " TUMBLING ( SIZE 30 SECONDS ) " + }, + "selectExpressions" : [ "KSQL_INTERNAL_COL_0 AS ID", "(WINDOWSTART / 2) AS KSQL_COL_0", "(WINDOWEND / KSQL_INTERNAL_COL_0) AS KSQL_COL_1", "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent7909833806392228816", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.authentication.plugin.class" : null, + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.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.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/spec.json new file mode 100644 index 000000000000..9884e2fc8afc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/spec.json @@ -0,0 +1,72 @@ +{ + "version" : "6.0.0", + "timestamp" : 1584639516855, + "schemas" : { + "CTAS_S2_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_S2_0.S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 2, + "value" : "2", + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : 100, + "value" : "100", + "timestamp" : 2000 + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2", + "timestamp" : 4999 + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "2", + "timestamp" : 5000 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 2, + "value" : "2,0,15000,1", + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : "100,0,300,1", + "timestamp" : 2000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 2, + "value" : "2,0,15000,2", + "timestamp" : 4999, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + }, { + "topic" : "S2", + "key" : 2, + "value" : "2,0,15000,3", + "timestamp" : 5000, + "window" : { + "start" : 0, + "end" : 30000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/topology new file mode 100644 index 000000000000..9f56837d5dec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/window-bounds_-_in_expressions/6.0.0_1584639516855/topology @@ -0,0 +1,28 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Aggregate-WindowSelect + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Aggregate-WindowSelect (stores: []) + --> Aggregate-Project + <-- Aggregate-Aggregate-ToOutputSchema + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000007 + <-- Aggregate-Aggregate-WindowSelect + Processor: KTABLE-TOSTREAM-0000000007 (stores: []) + --> KSTREAM-SINK-0000000008 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000008 (topic: S2) + <-- KTABLE-TOSTREAM-0000000007 + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json index 10bd4d780786..085c2f56bdcd 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 @@ -876,6 +876,38 @@ "type": "io.confluent.ksql.util.KsqlStatementException", "message": "'ROWKEY' is a reserved column name. It can only be used for KEY columns." } + }, + { + "name": "should handle sources with generated column names", + "statements": [ + "CREATE STREAM INPUT (KSQL_COL_3 INT KEY, KSQL_COL_2 INT, KSQL_COL_4 INT) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT KSQL_COL_2, ABS(KSQL_COL_2), ABS(KSQL_COL_2), ABS(KSQL_COL_2) FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "input", "key": 1, "value": {"KSQL_COL_2": 2, "KSQL_COL_4": 4}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"KSQL_COL_2": 2, "KSQL_COL_0": 2, "KSQL_COL_1": 2, "KSQL_COL_5": 2}} + ] + }, + { + "name": "should handle sources with generated column names - with select star", + "statements": [ + "CREATE STREAM INPUT (KSQL_COL_3 INT KEY, KSQL_COL_2 INT, KSQL_COL_4 INT) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT ABS(KSQL_COL_2), ABS(KSQL_COL_2), ABS(KSQL_COL_2), * FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "input", "key": 1, "value": {"KSQL_COL_2": 2, "KSQL_COL_4": 4}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"KSQL_COL_2": 2, "KSQL_COL_4": 4, "KSQL_COL_0": 2, "KSQL_COL_1": 2, "KSQL_COL_5": 2}} + ] } ] } 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 1546c312c97b..5efa5238fc27 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 @@ -30,7 +30,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -65,15 +65,15 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 3, "KSQL_AGG_VARIABLE_0": 2}, "timestamp": 3}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d2", "value": {"KSQL_INTERNAL_COL_0": "d2", "KSQL_INTERNAL_COL_1": 4, "KSQL_AGG_VARIABLE_0": 2}, "timestamp": 4}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 5, "KSQL_AGG_VARIABLE_0": 3}, "timestamp": 5}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":1}, "timestamp": 1}, - {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_1":1}, "timestamp": 2}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":2}, "timestamp": 3}, - {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_1":2}, "timestamp": 4}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":3}, "timestamp": 5} + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":1}, "timestamp": 1}, + {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_0":1}, "timestamp": 2}, + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":2}, "timestamp": 3}, + {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_0":2}, "timestamp": 4}, + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":3}, "timestamp": 5} ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -96,15 +96,15 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 1, "value": {"KSQL_INTERNAL_COL_0": 1, "KSQL_INTERNAL_COL_1": 3, "KSQL_AGG_VARIABLE_0": 2}, "timestamp": 3}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 2, "value": {"KSQL_INTERNAL_COL_0": 2, "KSQL_INTERNAL_COL_1": 4, "KSQL_AGG_VARIABLE_0": 2}, "timestamp": 4}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": 1, "value": {"KSQL_INTERNAL_COL_0": 1, "KSQL_INTERNAL_COL_1": 5, "KSQL_AGG_VARIABLE_0": 3}, "timestamp": 5}, - {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_1":1}, "timestamp": 1}, - {"topic": "OUTPUT", "key": 2, "value": {"ID": 2, "KSQL_COL_1":1}, "timestamp": 2}, - {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_1":2}, "timestamp": 3}, - {"topic": "OUTPUT", "key": 2, "value": {"ID": 2, "KSQL_COL_1":2}, "timestamp": 4}, - {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_1":3}, "timestamp": 5} + {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_0":1}, "timestamp": 1}, + {"topic": "OUTPUT", "key": 2, "value": {"ID": 2, "KSQL_COL_0":1}, "timestamp": 2}, + {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_0":2}, "timestamp": 3}, + {"topic": "OUTPUT", "key": 2, "value": {"ID": 2, "KSQL_COL_0":2}, "timestamp": 4}, + {"topic": "OUTPUT", "key": 1, "value": {"ID": 1, "KSQL_COL_0":3}, "timestamp": 5} ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY INT KEY, ID INT, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY INT KEY, ID INT, KSQL_COL_0 BIGINT"} ] } }, @@ -135,7 +135,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_2 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -157,7 +157,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, KSQL_COL_0 INTEGER, KSQL_COL_1 BIGINT"} ] } }, @@ -181,15 +181,15 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "a|+|1", "value": {"KSQL_INTERNAL_COL_0": 1, "KSQL_INTERNAL_COL_1": "a", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 2}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "b|+|2", "value": {"KSQL_INTERNAL_COL_0": 2, "KSQL_INTERNAL_COL_1": "b", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 2}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "a|+|3", "value": {"KSQL_INTERNAL_COL_0": 3, "KSQL_INTERNAL_COL_1": "a", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 1}}, - {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_2": 1}}, - {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_2": 1}}, - {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_2": 2}}, - {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_2": 2}}, - {"topic": "OUTPUT", "key": "a|+|3", "value": {"F1": 3, "F2": "a", "KSQL_COL_2": 1}} + {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_0": 2}}, + {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_0": 2}}, + {"topic": "OUTPUT", "key": "a|+|3", "value": {"F1": 3, "F2": "a", "KSQL_COL_0": 1}} ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_2 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -215,7 +215,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_2 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -250,7 +250,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_2 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -276,17 +276,17 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "b|+|2", "value": {"KSQL_INTERNAL_COL_0": 2, "KSQL_INTERNAL_COL_1": "b", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 0}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "b|+|1", "value": {"KSQL_INTERNAL_COL_0": 1, "KSQL_INTERNAL_COL_1": "b", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 0}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "a|+|1", "value": {"KSQL_INTERNAL_COL_0": 1, "KSQL_INTERNAL_COL_1": "a", "KSQL_INTERNAL_COL_2": 0, "KSQL_AGG_VARIABLE_0": 1}}, - {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_2": 1}}, - {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_2": 1}}, - {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_2": 0}}, - {"topic": "OUTPUT", "key": "b|+|1", "value": {"F1": 1, "F2": "b", "KSQL_COL_2": 1}}, - {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_2": 0}}, - {"topic": "OUTPUT", "key": "b|+|1", "value": {"F1": 1, "F2": "b", "KSQL_COL_2": 0}}, - {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_2": 1}} + {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_0": 0}}, + {"topic": "OUTPUT", "key": "b|+|1", "value": {"F1": 1, "F2": "b", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "b|+|2", "value": {"F1": 2, "F2": "b", "KSQL_COL_0": 0}}, + {"topic": "OUTPUT", "key": "b|+|1", "value": {"F1": 1, "F2": "b", "KSQL_COL_0": 0}}, + {"topic": "OUTPUT", "key": "a|+|1", "value": {"F1": 1, "F2": "a", "KSQL_COL_0": 1}} ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_2 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, F1 INT, F2 STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -322,7 +322,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -358,7 +358,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY DOUBLE KEY, DATA DOUBLE, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY DOUBLE KEY, DATA DOUBLE, KSQL_COL_0 BIGINT"} ] } }, @@ -387,15 +387,15 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 0, "KSQL_AGG_VARIABLE_0": 2}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d2", "value": {"KSQL_INTERNAL_COL_0": "d2", "KSQL_INTERNAL_COL_1": 0, "KSQL_AGG_VARIABLE_0": 2}}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 0, "KSQL_AGG_VARIABLE_0": 3}}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":1}}, - {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_1":1}}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":2}}, - {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_1":2}}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1":3}} + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":1}}, + {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_0":1}}, + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":2}}, + {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_0":2}}, + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0":3}} ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -444,7 +444,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_1 BIGINT"} + {"name": "OUTPUT", "type": "table", "schema": "ROWKEY STRING KEY, DATA STRING, KSQL_COL_0 BIGINT"} ] } }, @@ -1198,9 +1198,9 @@ {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 1, "KSQL_AGG_VARIABLE_0": 1, "KSQL_AGG_VARIABLE_1": 1}, "timestamp": 1}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d2", "value": {"KSQL_INTERNAL_COL_0": "d2", "KSQL_INTERNAL_COL_1": 2, "KSQL_AGG_VARIABLE_0": 1, "KSQL_AGG_VARIABLE_1": 1}, "timestamp": 2}, {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "key": "d1", "value": {"KSQL_INTERNAL_COL_0": "d1", "KSQL_INTERNAL_COL_1": 3, "KSQL_AGG_VARIABLE_0": 2, "KSQL_AGG_VARIABLE_1": 2}, "timestamp": 3}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1": 1, "KSQL_COL_2": 1, "COPY": "d1"}, "timestamp": 1}, - {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_1": 1, "KSQL_COL_2": 1, "COPY": "d2"}, "timestamp": 2}, - {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_1": 2, "KSQL_COL_2": 2, "COPY": "d1"}, "timestamp": 3} + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0": 1, "KSQL_COL_1": 1, "COPY": "d1"}, "timestamp": 1}, + {"topic": "OUTPUT", "key": "d2", "value": {"DATA": "d2", "KSQL_COL_0": 1, "KSQL_COL_1": 1, "COPY": "d2"}, "timestamp": 2}, + {"topic": "OUTPUT", "key": "d1", "value": {"DATA": "d1", "KSQL_COL_0": 2, "KSQL_COL_1": 2, "COPY": "d1"}, "timestamp": 3} ] }, { 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 74d50612f904..ba0cc3faabe9 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 @@ -1305,7 +1305,7 @@ {"topic": "right_topic", "key": 1, "value": {"KSQL_COL_3": 3}, "timestamp": 10} ], "outputs": [ - {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", "key": 1, "value": {"TT_KSQL_COL_3": 3, "TT_ROWTIME": 10, "TT_KSQL_COL_1": 1, "TT_KSQL_COL_4": 1}, "timestamp": 10}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", "key": 1, "value": {"TT_KSQL_COL_3": 3, "TT_ROWTIME": 10, "TT_KSQL_COL_1": 1, "TT_KSQL_COL_0": 1}, "timestamp": 10}, {"topic": "OUTPUT", "key": 1, "value": {"KSQL_COL_2": 2}, "timestamp": 10} ] }, diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-field.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-field.json index a34e4b6b1c9c..afedaf2b1fce 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-field.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-field.json @@ -124,7 +124,7 @@ {"topic": "input_topic", "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"FOO":1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"FOO":1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -133,7 +133,7 @@ "name": "OUTPUT", "type": "table", "keyField": "FOO", - "schema": "ROWKEY INT KEY, FOO INT, KSQL_COL_1 BIGINT" + "schema": "ROWKEY INT KEY, FOO INT, KSQL_COL_0 BIGINT" } ] } @@ -148,7 +148,7 @@ {"topic": "input_topic", "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"ALIASED":1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"ALIASED":1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -366,7 +366,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"FOO":1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"FOO":1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -388,7 +388,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"ALIASED":1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"ALIASED":1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -431,7 +431,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 2, "value": {"BAR":2, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 2, "value": {"BAR":2, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -449,7 +449,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 2, "value": {"ALIASED":2, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 2, "value": {"ALIASED":2, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -504,7 +504,7 @@ {"topic": "input_topic", "key": "x", "value": {"foo": 1}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"FOO": 1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"FOO": 1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -526,7 +526,7 @@ {"topic": "input_topic", "key": "x", "value": {"foo": 1}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1, "value": {"ALIASED": 1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1, "value": {"ALIASED": 1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -601,7 +601,7 @@ {"topic": "input_topic", "key": 1,"value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1,"value": {"FOO": 1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1,"value": {"FOO": 1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -619,7 +619,7 @@ {"topic": "input_topic", "key": 1,"value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 1,"value": {"ALIASED": 1, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 1,"value": {"ALIASED": 1, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -655,7 +655,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 2, "value": {"BAR": 2, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 2, "value": {"BAR": 2, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -673,7 +673,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key": 2,"value": {"ALIASED": 2, "KSQL_COL_1": 1}} + {"topic": "OUTPUT", "key": 2,"value": {"ALIASED": 2, "KSQL_COL_0": 1}} ], "post": { "sources": [ @@ -741,7 +741,7 @@ {"topic": "input_topic", "key": 1, "value": {"foo": 1, "bar": 2}} ], "outputs": [ - {"topic": "OUTPUT", "key":"2|+|1", "value": {"FOO": 1, "BAR": 2, "KSQL_COL_2": 1}} + {"topic": "OUTPUT", "key":"2|+|1", "value": {"FOO": 1, "BAR": 2, "KSQL_COL_0": 1}} ], "post": { "sources": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-schemas.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-schemas.json index a8e5ce7a6864..e1ac55f0e35f 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-schemas.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/key-schemas.json @@ -440,7 +440,7 @@ {"topic": "input", "key": 10, "value": {"value": 1}} ], "outputs": [ - {"topic": "OUTPUT", "key": 10, "value": {"EXP": 22026.465794806718, "KSQL_COL_1": 1}, "window": {"start": 0, "end": 30000, "type": "time"}} + {"topic": "OUTPUT", "key": 10, "value": {"EXP": 22026.465794806718, "KSQL_COL_0": 1}, "window": {"start": 0, "end": 30000, "type": "time"}} ], "post": { "sources": [ @@ -448,7 +448,7 @@ "name": "OUTPUT", "type": "table", "keyFormat": {"format": "KAFKA", "windowType": "TUMBLING", "windowSize": 30000}, - "schema": "`ROWKEY` BIGINT KEY, `EXP` DOUBLE, `KSQL_COL_1` BIGINT" + "schema": "`ROWKEY` BIGINT KEY, `EXP` DOUBLE, `KSQL_COL_0` BIGINT" } ] } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/literals.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/literals.json index a009f9b86050..ec378656abb8 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/literals.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/literals.json @@ -67,7 +67,7 @@ ], "post": { "sources": [ - {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, ID BIGINT, KSQL_COL_1 DECIMAL(4,3)"} + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, ID BIGINT, KSQL_COL_0 DECIMAL(4,3)"} ] } }, diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/simple-struct.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/simple-struct.json index 994efa11ce3f..b3e27ea12026 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/simple-struct.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/simple-struct.json @@ -591,7 +591,7 @@ "timestamp": 0, "value": { "ITEMID": "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Food\"},\"NAME\":\"Item_6\"}", - "KSQL_COL_1": "6" + "KSQL_COL_0": "6" }, "key": "0" }, @@ -600,7 +600,7 @@ "timestamp": 0, "value": { "ITEMID": "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Produce\"},\"NAME\":\"Item_6\"}", - "KSQL_COL_1": "6" + "KSQL_COL_0": "6" }, "key": "100" }, @@ -609,7 +609,7 @@ "timestamp": 0, "value": { "ITEMID": "{\"ITEMID\":6,\"CATEGORY\":{\"ID\":2,\"NAME\":\"Produce\"},\"NAME\":\"Item_6\"}", - "KSQL_COL_1": "6" + "KSQL_COL_0": "6" }, "key": "101" }, @@ -618,7 +618,7 @@ "timestamp": 0, "value": { "ITEMID": "{\"ITEMID\":2,\"CATEGORY\":{\"ID\":1,\"NAME\":\"Food\"},\"NAME\":\"Item_2\"}", - "KSQL_COL_1": "2" + "KSQL_COL_0": "2" }, "key": "101" }, @@ -627,7 +627,7 @@ "timestamp": 0, "value": { "ITEMID": "{\"ITEMID\":5,\"CATEGORY\":{\"ID\":1,\"NAME\":\"Produce\"},\"NAME\":\"Item_5\"}", - "KSQL_COL_1": "5" + "KSQL_COL_0": "5" }, "key": "101" } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/string.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/string.json index dc535a730633..10a2b7860a12 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/string.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/string.json @@ -29,7 +29,7 @@ { "name": "OUTPUT", "type": "stream", - "schema": "`ROWKEY` STRING KEY, `TEXT` STRING, `KSQL_COL_1` BOOLEAN" + "schema": "ROWKEY STRING KEY, TEXT STRING, KSQL_COL_0 BOOLEAN" } ] } diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/table-functions.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/table-functions.json index d19f9fd1b114..69d7c14992bb 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/table-functions.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/table-functions.json @@ -126,9 +126,9 @@ {"topic": "test_topic", "key": "0", "value": {"ID": 0, "F0": 1, "F1": 10, "F2": 11, "F3": 12}} ], "outputs": [ - {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_1": 21}}, - {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_1": 23}}, - {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_1": 22}} + {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_0": 21}}, + {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_0": 23}}, + {"topic": "OUTPUT", "key": "0", "value": {"F0": 1, "KSQL_COL_0": 22}} ] }, { @@ -333,8 +333,8 @@ {"topic": "test_topic", "key": "c", "value": {"shouldThrow": false}} ], "outputs": [ - {"topic": "OUTPUT", "key": "a", "value": {"SHOULDTHROW": false, "KSQL_COL_1": false}}, - {"topic": "OUTPUT", "key": "c", "value": {"SHOULDTHROW": false, "KSQL_COL_1": false}} + {"topic": "OUTPUT", "key": "a", "value": {"SHOULDTHROW": false, "KSQL_COL_0": false}}, + {"topic": "OUTPUT", "key": "c", "value": {"SHOULDTHROW": false, "KSQL_COL_0": false}} ] }, { @@ -349,8 +349,8 @@ {"topic": "test_topic", "key": "c", "value": {"shouldThrow": false}} ], "outputs": [ - {"topic": "OUTPUT", "key": "a", "value": {"SHOULDTHROW": false, "KSQL_COL_1": 0}}, - {"topic": "OUTPUT", "key": "c", "value": {"SHOULDTHROW": false, "KSQL_COL_1": 0}} + {"topic": "OUTPUT", "key": "a", "value": {"SHOULDTHROW": false, "KSQL_COL_0": 0}}, + {"topic": "OUTPUT", "key": "c", "value": {"SHOULDTHROW": false, "KSQL_COL_0": 0}} ] } ] diff --git a/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json b/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json index 1c18a728e703..0085935c4d70 100644 --- a/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json +++ b/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json @@ -503,7 +503,7 @@ {"admin": {"@type": "currentStatus"}}, {"admin": {"@type": "currentStatus"}}, {"query": [ - {"header":{"schema":"`COUNT` BIGINT, `ID` STRING, `KSQL_COL_2` BIGINT"}}, + {"header":{"schema":"`COUNT` BIGINT, `ID` STRING, `KSQL_COL_0` BIGINT"}}, {"row":{"columns":[1, "10x", 2]}} ]} ] @@ -524,7 +524,7 @@ {"admin": {"@type": "currentStatus"}}, {"admin": {"@type": "currentStatus"}}, {"query": [ - {"header":{"schema":"`COUNT` BIGINT, `ID` STRING, `KSQL_COL_2` BIGINT"}}, + {"header":{"schema":"`COUNT` BIGINT, `ID` STRING, `KSQL_COL_0` BIGINT"}}, {"row":{"columns":[1, "10x", 2]}} ]} ] diff --git a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/PartitionByParamsFactory.java b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/PartitionByParamsFactory.java index 0c40818b586d..2ee0c5fad463 100644 --- a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/PartitionByParamsFactory.java +++ b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/PartitionByParamsFactory.java @@ -35,6 +35,7 @@ import java.util.Optional; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.stream.Stream; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.KeyValue; @@ -112,7 +113,7 @@ private static LogicalSchema buildSchema( final ColumnName newKeyName = partitionByCol .map(Column::name) - .orElseGet(() -> ColumnNames.nextGeneratedColumnAlias(sourceSchema)); + .orElseGet(() -> ColumnNames.columnAliasGenerator(Stream.of(sourceSchema)).get()); final Builder builder = LogicalSchema.builder() .withRowTime() diff --git a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/AggregateParamsFactoryTest.java b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/AggregateParamsFactoryTest.java index d566ff96dbbf..256d6794a367 100644 --- a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/AggregateParamsFactoryTest.java +++ b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/AggregateParamsFactoryTest.java @@ -19,7 +19,6 @@ import io.confluent.ksql.execution.function.udaf.KudafUndoAggregator; import io.confluent.ksql.execution.streams.AggregateParamsFactory.KudafAggregatorFactory; import io.confluent.ksql.execution.streams.AggregateParamsFactory.KudafUndoAggregatorFactory; -import io.confluent.ksql.execution.transform.KsqlProcessingContext; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.KsqlAggregateFunction; import io.confluent.ksql.name.ColumnName; @@ -66,10 +65,6 @@ public class AggregateParamsFactoryTest { FunctionName.of("TABLE_AGG"), ImmutableList.of(new UnqualifiedColumnReferenceExp(ColumnName.of("ARGUMENT0"))) ); - private static final FunctionCall WINDOW_START = new FunctionCall( - FunctionName.of("WindowStart"), - ImmutableList.of(new UnqualifiedColumnReferenceExp(ColumnName.of("ARGUMENT0"))) - ); private static final String INITIAL_VALUE1 = "initial"; private static final List FUNCTIONS = ImmutableList.of(AGG0, AGG1); @@ -82,8 +77,6 @@ public class AggregateParamsFactoryTest { @Mock private TableAggregationFunction tableAgg; @Mock - private KsqlAggregateFunction windowStart; - @Mock private KudafAggregatorFactory udafFactory; @Mock private KudafUndoAggregatorFactory undoUdafFactory; @@ -91,8 +84,6 @@ public class AggregateParamsFactoryTest { private KudafAggregator aggregator; @Mock private KudafUndoAggregator undoAggregator; - @Mock - private KsqlProcessingContext ctx; private AggregateParams aggregateParams; @@ -102,13 +93,11 @@ public void init() { when(functionRegistry.getAggregateFunction(same(AGG0.getName()), any(), any())) .thenReturn(agg0); when(agg0.getInitialValueSupplier()).thenReturn(() -> INITIAL_VALUE0); - when(agg0.name()).thenReturn(AGG0.getName()); when(agg0.returnType()).thenReturn(SqlTypes.INTEGER); when(agg0.getAggregateType()).thenReturn(SqlTypes.BIGINT); when(functionRegistry.getAggregateFunction(same(AGG1.getName()), any(), any())) .thenReturn(agg1); when(agg1.getInitialValueSupplier()).thenReturn(() -> INITIAL_VALUE1); - when(agg1.name()).thenReturn(AGG1.getName()); when(agg1.returnType()).thenReturn(SqlTypes.STRING); when(agg1.getAggregateType()).thenReturn(SqlTypes.DOUBLE); when(functionRegistry.getAggregateFunction(same(TABLE_AGG.getName()), any(), any())) @@ -116,13 +105,6 @@ public void init() { when(tableAgg.getInitialValueSupplier()).thenReturn(() -> INITIAL_VALUE0); when(tableAgg.returnType()).thenReturn(SqlTypes.INTEGER); when(tableAgg.getAggregateType()).thenReturn(SqlTypes.BIGINT); - when(tableAgg.name()).thenReturn(TABLE_AGG.getName()); - when(functionRegistry.getAggregateFunction(same(WINDOW_START.getName()), any(), any())) - .thenReturn(windowStart); - when(windowStart.getInitialValueSupplier()).thenReturn(() -> INITIAL_VALUE0); - when(windowStart.name()).thenReturn(WINDOW_START.getName()); - when(windowStart.returnType()).thenReturn(SqlTypes.BIGINT); - when(windowStart.getAggregateType()).thenReturn(SqlTypes.BIGINT); when(udafFactory.create(anyInt(), any())).thenReturn(aggregator); when(undoUdafFactory.create(anyInt(), any())).thenReturn(undoAggregator); diff --git a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/PartitionByParamsFactoryTest.java b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/PartitionByParamsFactoryTest.java index 6c1d24d9e016..c85bd30ad029 100644 --- a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/PartitionByParamsFactoryTest.java +++ b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/PartitionByParamsFactoryTest.java @@ -38,7 +38,6 @@ import io.confluent.ksql.function.udf.Kudf; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.name.ColumnNames; import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; @@ -164,12 +163,12 @@ public void shouldBuildResultSchemaWhenPartitioningByOtherExpressionType() { // Then: assertThat(resultSchema, is(LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("KSQL_COL_4"), SqlTypes.INTEGER) + .keyColumn(ColumnName.of("KSQL_COL_0"), SqlTypes.INTEGER) .valueColumn(COL1, SqlTypes.INTEGER) .valueColumn(COL2, SqlTypes.INTEGER) .valueColumn(SchemaUtil.ROWTIME_NAME, SqlTypes.BIGINT) .valueColumn(COL0, SqlTypes.STRING) - .valueColumn(ColumnName.of("KSQL_COL_4"), SqlTypes.INTEGER) + .valueColumn(ColumnName.of("KSQL_COL_0"), SqlTypes.INTEGER) .build())); } @@ -197,7 +196,7 @@ public void shouldPartitionByNullAnyRowsWhereFailedToExtractKey() { // Then: final KeyBuilder keyBuilder = StructKeyUtil - .keyBuilder(ColumnNames.generatedColumnAlias(4), SqlTypes.INTEGER); + .keyBuilder(ColumnName.of("KSQL_COL_0"), SqlTypes.INTEGER); assertThat(result.key, is(keyBuilder.build(null))); }