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