From eb96b68042aa9a0daf1ac9a01989407b0f34dfee Mon Sep 17 00:00:00 2001 From: Vicky Papavasileiou Date: Fri, 21 Aug 2020 09:56:38 -0700 Subject: [PATCH] feat: Support subscript and nested functions in grouping queries (#5998) * fixed handling subscript and nested functions * remove intermediate topics from test file * addressed comments, handle struct and arithmetic, added plans * fixed window bounds error * adding historic plans * addressed Almog's comments * minor fixes --- .../ksql/analyzer/AggregateAnalyzer.java | 166 +++++++++------- .../6.1.0_1597866556719/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556719/spec.json | 158 +++++++++++++++ .../6.1.0_1597866556719/topology | 40 ++++ .../6.1.0_1597889438730/plan.json | 179 +++++++++++++++++ .../6.1.0_1597889438730/spec.json | 153 ++++++++++++++ .../6.1.0_1597889438730/topology | 40 ++++ .../6.1.0_1597866556623/plan.json | 186 ++++++++++++++++++ .../6.1.0_1597866556623/spec.json | 154 +++++++++++++++ .../6.1.0_1597866556623/topology | 49 +++++ .../6.1.0_1597866556426/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556426/spec.json | 162 +++++++++++++++ .../6.1.0_1597866556426/topology | 40 ++++ .../6.1.0_1597866556536/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556536/spec.json | 162 +++++++++++++++ .../6.1.0_1597866556536/topology | 40 ++++ .../6.1.0_1597866556344/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556344/spec.json | 172 ++++++++++++++++ .../6.1.0_1597866556344/topology | 40 ++++ .../6.1.0_1597866556171/plan.json | 186 ++++++++++++++++++ .../6.1.0_1597866556171/spec.json | 150 ++++++++++++++ .../6.1.0_1597866556171/topology | 49 +++++ .../6.1.0_1597866556241/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556241/spec.json | 154 +++++++++++++++ .../6.1.0_1597866556241/topology | 40 ++++ .../6.1.0_1597866556081/plan.json | 179 +++++++++++++++++ .../6.1.0_1597866556081/spec.json | 152 ++++++++++++++ .../6.1.0_1597866556081/topology | 40 ++++ .../query-validation-tests/group-by.json | 181 ++++++++++++++++- 29 files changed, 3694 insertions(+), 73 deletions(-) create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/topology diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/AggregateAnalyzer.java b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/AggregateAnalyzer.java index ef10833e318e..4022b0566d08 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/AggregateAnalyzer.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/AggregateAnalyzer.java @@ -69,16 +69,13 @@ private static final class AggAnalyzer { private final FunctionRegistry functionRegistry; private final Set groupBy; - // The list of columns from the source schema that are used in aggregate columns, but not as - // parameters to the aggregate functions and which are not part of the GROUP BY clause: - private final List aggSelectsNotPartOfGroupBy = new ArrayList<>(); - - // The list of non-aggregate select expression which are not part of the GROUP BY clause: + // The list of expressions that appear in the SELECT clause outside of aggregate functions. + // Used for throwing an error if these columns are not part of the GROUP BY clause. private final List nonAggSelectsNotPartOfGroupBy = new ArrayList<>(); // The list of columns from the source schema that are used in the HAVING clause outside // of aggregate functions which are not part of the GROUP BY clause: - private final List nonAggHavingNotPartOfGroupBy = new ArrayList<>(); + private final List nonAggHavingNotPartOfGroupBy = new ArrayList<>(); AggAnalyzer( final ImmutableAnalysis analysis, @@ -107,69 +104,78 @@ public void process(final List finalProjection) { } private void processSelect(final Expression expression) { - final Set nonAggParams = new HashSet<>(); - final AggregateVisitor visitor = new AggregateVisitor(this, (aggFuncName, node) -> { - if (aggFuncName.isPresent()) { - throwOnWindowBoundColumnIfWindowedAggregate(node); - } else { - nonAggParams.add(node); - } - }); + final Set nonAggParams = new HashSet<>(); + final AggregateVisitor visitor = new AggregateVisitor( + this, + groupBy, + (aggFuncName, node) -> { + if (aggFuncName.isPresent()) { + throwOnWindowBoundColumnIfWindowedAggregate(node); + } else { + if (!groupBy.contains(node)) { + nonAggParams.add(node); + } + } + }); visitor.process(expression, null); - - if (visitor.visitedAggFunction) { - captureAggregateSelectNotPartOfGroupBy(nonAggParams); - } else { - captureNonAggregateSelectNotPartOfGroupBy(expression, nonAggParams); - } - + captureNonAggregateSelectNotPartOfGroupBy(expression, nonAggParams); aggregateAnalysis.addFinalSelectExpression(expression); } private void processGroupBy(final Expression expression) { - final AggregateVisitor visitor = new AggregateVisitor(this, (aggFuncName, node) -> { - if (aggFuncName.isPresent()) { - throw new KsqlException("GROUP BY does not support aggregate functions: " - + aggFuncName.get().text() + " is an aggregate function."); - } - throwOnWindowBoundColumnIfWindowedAggregate(node); - }); + final AggregateVisitor visitor = new AggregateVisitor( + this, + groupBy, + (aggFuncName, node) -> { + if (aggFuncName.isPresent()) { + throw new KsqlException("GROUP BY does not support aggregate functions: " + + aggFuncName.get().text() + " is an aggregate function."); + } + throwOnWindowBoundColumnIfWindowedAggregate(node); + }); visitor.process(expression, null); } private void processWhere(final Expression expression) { - final AggregateVisitor visitor = new AggregateVisitor(this, (aggFuncName, node) -> - throwOnWindowBoundColumnIfWindowedAggregate(node)); + final AggregateVisitor visitor = new AggregateVisitor( + this, + groupBy, + (aggFuncName, node) -> + throwOnWindowBoundColumnIfWindowedAggregate(node)); visitor.process(expression, null); } private void processHaving(final Expression expression) { - final AggregateVisitor visitor = new AggregateVisitor(this, (aggFuncName, node) -> { - throwOnWindowBoundColumnIfWindowedAggregate(node); + final AggregateVisitor visitor = new AggregateVisitor( + this, + groupBy, + (aggFuncName, node) -> { + throwOnWindowBoundColumnIfWindowedAggregate(node); - if (!aggFuncName.isPresent()) { - captureNoneAggregateHavingNotPartOfGroupBy(node); - } - }); + if (!aggFuncName.isPresent()) { + captureNonAggregateHavingNotPartOfGroupBy(node); + } + }); visitor.process(expression, null); aggregateAnalysis.setHavingExpression(expression); } - private void throwOnWindowBoundColumnIfWindowedAggregate( - final ColumnReferenceExp node - ) { + private void throwOnWindowBoundColumnIfWindowedAggregate(final Expression node) { // Window bounds are supported for operations on windowed sources if (!analysis.getWindowExpression().isPresent()) { return; } + if (!(node instanceof ColumnReferenceExp)) { + return; + } // For non-windowed sources, with a windowed GROUP BY, they are only supported in selects: - if (SystemColumns.isWindowBound(node.getColumnName())) { + if (SystemColumns.isWindowBound(((ColumnReferenceExp)node).getColumnName())) { throw new KsqlException( "Window bounds column " + node + " can only be used in the SELECT clause of " + "windowed aggregations and can not be passed to aggregate functions." @@ -204,7 +210,7 @@ private static Set getGroupByExpressions( private void captureNonAggregateSelectNotPartOfGroupBy( final Expression expression, - final Set nonAggParams + final Set nonAggParams ) { final boolean matchesGroupBy = groupBy.contains(expression); if (matchesGroupBy) { @@ -220,16 +226,8 @@ private void captureNonAggregateSelectNotPartOfGroupBy( nonAggSelectsNotPartOfGroupBy.add(expression); } - private void captureAggregateSelectNotPartOfGroupBy( - final Set nonAggParams - ) { - nonAggParams.stream() - .filter(param -> !groupBy.contains(param)) - .forEach(aggSelectsNotPartOfGroupBy::add); - } - - private void captureNoneAggregateHavingNotPartOfGroupBy(final ColumnReferenceExp nonAggColumn) { - if (groupBy.contains(new UnqualifiedColumnReferenceExp(nonAggColumn.getColumnName()))) { + private void captureNonAggregateHavingNotPartOfGroupBy(final Expression nonAggColumn) { + if (groupBy.contains(nonAggColumn)) { return; } @@ -256,21 +254,7 @@ private void enforceAggregateRules() { "Non-aggregate SELECT expression(s) not part of GROUP BY: " + unmatchedSelects + System.lineSeparator() - + "Either add the column to the GROUP BY or remove it from the SELECT." - ); - } - - final String unmatchedSelectsAgg = aggSelectsNotPartOfGroupBy.stream() - .map(Objects::toString) - .collect(Collectors.joining(", ")); - - if (!unmatchedSelectsAgg.isEmpty()) { - throw new KsqlException( - "Column used in aggregate SELECT expression(s) outside of aggregate functions " - + "not part of GROUP BY: " - + unmatchedSelectsAgg - + System.lineSeparator() - + "Either add the column to the GROUP BY or remove it from the SELECT." + + "Either add the column(s) to the GROUP BY or remove them from the SELECT." ); } @@ -288,26 +272,60 @@ private void enforceAggregateRules() { } } + /** + * This visitor performs two tasks: Create the input schema to the AggregateNode and validations. + * + *

For creating the input schema, it checks if any expression along the path from the root + * expression to the leaf (UnqualifiedColumnReference) is part of the groupBy. If at least one is, + * then the UnqualifiedColumnReference is added to the schema. + * + *

For validation, the visitor checks that: + *

    + *
  1. expressions not in aggregate functions are part of the grouping clause
  2. + *
  3. aggregate functions are not nested
  4. + *
  5. window clauses (windowstart, windowend) don't appear in aggregate functions or + * groupBy
  6. + *
  7. aggregate functions don't appear in the groupBy clause
  8. + *
  9. expressions in the having clause are either aggregate functions or grouping keys
  10. + *
+ */ private static final class AggregateVisitor extends TraversalExpressionVisitor { - private final BiConsumer, ColumnReferenceExp> dereferenceCollector; + private final BiConsumer, Expression> dereferenceCollector; private final ColumnReferenceExp defaultArgument; private final MutableAggregateAnalysis aggregateAnalysis; private final FunctionRegistry functionRegistry; + private final Set groupBy; + private Expression currentlyInExpressionPartOfGroupBy; private Optional aggFunctionName = Optional.empty(); - private boolean visitedAggFunction = false; + private boolean currentlyInAggregateFunction = false; private AggregateVisitor( final AggAnalyzer aggAnalyzer, - final BiConsumer, ColumnReferenceExp> dereferenceCollector + final Set groupBy, + final BiConsumer, Expression> dereferenceCollector ) { this.defaultArgument = aggAnalyzer.analysis.getDefaultArgument(); this.aggregateAnalysis = aggAnalyzer.aggregateAnalysis; this.functionRegistry = aggAnalyzer.functionRegistry; + this.groupBy = groupBy; this.dereferenceCollector = requireNonNull(dereferenceCollector, "dereferenceCollector"); } + @Override + public Void process(final Expression node, final Void context) { + if (groupBy.contains(node) && currentlyInExpressionPartOfGroupBy == null) { + currentlyInExpressionPartOfGroupBy = node; + } + super.process(node, context); + if (currentlyInExpressionPartOfGroupBy != null + && currentlyInExpressionPartOfGroupBy == node) { + currentlyInExpressionPartOfGroupBy = null; + } + return null; + } + @Override public Void visitFunctionCall(final FunctionCall node, final Void context) { final FunctionName functionName = node.getName(); @@ -323,7 +341,7 @@ public Void visitFunctionCall(final FunctionCall node, final Void context) { + aggFunctionName.get().text() + "(" + functionName.text() + "())"); } - visitedAggFunction = true; + currentlyInAggregateFunction = true; aggFunctionName = Optional.of(functionName); functionCall.getArguments().forEach(aggregateAnalysis::addAggregateFunctionArgument); @@ -344,9 +362,13 @@ public Void visitUnqualifiedColumnReference( final UnqualifiedColumnReferenceExp node, final Void context ) { - dereferenceCollector.accept(aggFunctionName, node); - + if (currentlyInExpressionPartOfGroupBy == null + || currentlyInAggregateFunction + || SystemColumns.isWindowBound(node.getColumnName())) { + dereferenceCollector.accept(aggFunctionName, node); + } if (!SystemColumns.isWindowBound(node.getColumnName())) { + // Used to infer the required columns in the INPUT schema of the aggregate node aggregateAnalysis.addRequiredColumn(node); } return null; diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/plan.json new file mode 100644 index 000000000000..9cf30259eef6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 INTEGER, COL2 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n (INPUT.COL1 + INPUT.COL2) G1,\n AS_VALUE((INPUT.COL1 + INPUT.COL2)) KSQL_COL_0,\n COUNT(*) KSQL_COL_1\nFROM INPUT INPUT\nGROUP BY (INPUT.COL1 + INPUT.COL2)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "COL2 AS COL2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "(COL1 + COL2)" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "COL2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "G1" ], + "selectExpressions" : [ "AS_VALUE((COL1 + COL2)) AS KSQL_COL_0", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/spec.json new file mode 100644 index 000000000000..9447708895c8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/spec.json @@ -0,0 +1,158 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556719, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "arithmetic in group by column used in non-aggregate function in select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : 1, + "col2" : 1 + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : 2, + "col2" : 2 + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : 3, + "col2" : 3 + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : 4, + "col2" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "KSQL_COL_0" : 2, + "KSQL_COL_1" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 4, + "value" : { + "KSQL_COL_0" : 4, + "KSQL_COL_1" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 6, + "value" : { + "KSQL_COL_0" : 6, + "KSQL_COL_1" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "KSQL_COL_0" : 8, + "KSQL_COL_1" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 INT, col2 INT) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1+col2 AS G1, AS_VALUE(col1+col2), COUNT(*) FROM input GROUP BY col1+col2;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` INTEGER, `COL2` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_arithmetic_in_group_by_column_used_in_non-aggregate_function_in_select/6.1.0_1597866556719/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_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/plan.json new file mode 100644 index 000000000000..486b0bf2fa20 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 MAP, COL2 MAP) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` MAP, `COL2` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n (INPUT.COL1['foo'] + INPUT.COL2['bar']) G1,\n AS_VALUE((INPUT.COL1['foo'] + INPUT.COL2['bar'])) KSQL_COL_0,\n COUNT(*) KSQL_COL_1\nFROM INPUT INPUT\nGROUP BY (INPUT.COL1['foo'] + INPUT.COL2['bar'])\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` MAP, `COL2` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "COL2 AS COL2", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "(COL1['foo'] + COL2['bar'])" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "COL2", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "G1" ], + "selectExpressions" : [ "AS_VALUE((COL1['foo'] + COL2['bar'])) AS KSQL_COL_0", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/spec.json new file mode 100644 index 000000000000..154505866066 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/spec.json @@ -0,0 +1,153 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597889438730, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` INTEGER KEY, `COL1` MAP, `COL2` MAP, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` MAP, `COL2` MAP", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` INTEGER KEY, `COL1` MAP, `COL2` MAP, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "expressions used in non-aggregate function in select whose children are not part of group-by", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : { + "a" : 1 + }, + "col2" : { + "b" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : { + "foo" : 1 + }, + "col2" : { + "bar" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : { + "bar" : 1 + }, + "col2" : { + "foo" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : { + "foo" : 1 + }, + "col2" : { + "foo" : 1 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "KSQL_COL_0" : 2, + "KSQL_COL_1" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 MAP, col2 MAP) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1['foo']+col2['bar'] AS G1, AS_VALUE(col1['foo']+col2['bar']), COUNT(*) FROM input GROUP BY col1['foo']+col2['bar'];" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` MAP, `COL2` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`G1` INTEGER KEY, `KSQL_COL_0` INTEGER, `KSQL_COL_1` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_expressions_used_in_non-aggregate_function_in_select_whose_children_are_not_part_of_group-by/6.1.0_1597889438730/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_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/plan.json new file mode 100644 index 000000000000..b0f05ff45b23 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/plan.json @@ -0,0 +1,186 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 STRING, COL2 STRING, COL3 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INITCAP(INPUT.COL1) G1,\n INPUT.COL2 G2,\n TRIM(INPUT.COL3) G3,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INITCAP(INPUT.COL1), INPUT.COL2, TRIM(INPUT.COL3)\nHAVING (SUBSTRING(TRIM(INPUT.COL3), 1, 4) = 'teen')\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "COL2 AS COL2", "COL3 AS COL3", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "INITCAP(COL1)", "COL2", "TRIM(COL3)" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "COL2", "COL3", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "filterExpression" : "(SUBSTRING(TRIM(COL3), 1, 4) = 'teen')" + }, + "keyColumnNames" : [ "KSQL_COL_1" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/spec.json new file mode 100644 index 000000000000..fc7597874f20 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/spec.json @@ -0,0 +1,154 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556623, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "function group by column used in non-aggregate function in having", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "teen spirit" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : "the", + "col2" : "man who", + "col3" : "stole the world" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "spring" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : " teen spirit " + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|teen spirit", + "value" : { + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "The|+|man who|+|stole the world", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|spring", + "value" : null + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|teen spirit", + "value" : { + "KSQL_COL_0" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, trim(COL3) AS G3, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, trim(col3) HAVING substring(trim(col3),1,4) = 'teen';" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`KSQL_COL_1` STRING KEY, `KSQL_COL_0` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_group_by_column_used_in_non-aggregate_function_in_having/6.1.0_1597866556623/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_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/plan.json new file mode 100644 index 000000000000..b50512d99422 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 STRING, COL2 STRING, COL3 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INITCAP(INPUT.COL1) G1,\n INPUT.COL2 G2,\n TRIM(INPUT.COL3) G3,\n CONCAT(INITCAP(INPUT.COL1), INPUT.COL2, TRIM(INPUT.COL3)) FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INITCAP(INPUT.COL1), INPUT.COL2, TRIM(INPUT.COL3)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "COL2 AS COL2", "COL3 AS COL3", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "INITCAP(COL1)", "COL2", "TRIM(COL3)" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "COL2", "COL3", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "KSQL_COL_1" ], + "selectExpressions" : [ "CONCAT(INITCAP(COL1), COL2, TRIM(COL3)) AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/spec.json new file mode 100644 index 000000000000..4f5da728d35e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556426, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "function in group-by and nested function in select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "teen spirit" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : "the", + "col2" : "man who", + "col3" : "stole the world" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "spring" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : " teen spirit " + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|teen spirit", + "value" : { + "FOO" : "Smellsliketeen spirit", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "The|+|man who|+|stole the world", + "value" : { + "FOO" : "Theman whostole the world", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|spring", + "value" : { + "FOO" : "Smellslikespring", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|teen spirit", + "value" : { + "FOO" : "Smellsliketeen spirit", + "KSQL_COL_0" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, TRIM(COL3) AS G3, concat(initcap(col1), col2, trim(col3)) AS foo, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, TRIM(col3);" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_function_in_group-by_and_nested_function_in_select/6.1.0_1597866556426/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_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/plan.json new file mode 100644 index 000000000000..1542184d1ed7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 STRING, COL2 STRING, COL3 STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INITCAP(INPUT.COL1) G1,\n INPUT.COL2 G2,\n INPUT.COL3 G3,\n CONCAT(INITCAP(INPUT.COL1), INPUT.COL2, TRIM(INPUT.COL3)) FOO,\n COUNT(*) KSQL_COL_0\nFROM INPUT INPUT\nGROUP BY INITCAP(INPUT.COL1), INPUT.COL2, INPUT.COL3\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "COL2 AS COL2", "COL3 AS COL3", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "INITCAP(COL1)", "COL2", "COL3" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "COL2", "COL3", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "KSQL_COL_1" ], + "selectExpressions" : [ "CONCAT(INITCAP(COL1), COL2, TRIM(COL3)) AS FOO", "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/spec.json new file mode 100644 index 000000000000..a15a71460fed --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556536, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "group by column in nested non-aggregate function in select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "teen spirit" + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : "the", + "col2" : "man who", + "col3" : "stole the world" + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : "spring" + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : "smells", + "col2" : "like", + "col3" : " teen spirit " + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|teen spirit", + "value" : { + "FOO" : "Smellsliketeen spirit", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "The|+|man who|+|stole the world", + "value" : { + "FOO" : "Theman whostole the world", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+|spring", + "value" : { + "FOO" : "Smellslikespring", + "KSQL_COL_0" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "Smells|+|like|+| teen spirit ", + "value" : { + "FOO" : "Smellsliketeen spirit", + "KSQL_COL_0" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, COL3 AS G3, concat(initcap(col1), col2, trim(col3)) AS foo, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, col3;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`KSQL_COL_1` STRING KEY, `FOO` STRING, `KSQL_COL_0` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` STRING, `COL2` STRING, `COL3` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_group_by_column_in_nested_non-aggregate_function_in_select/6.1.0_1597866556536/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_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/plan.json new file mode 100644 index 000000000000..3a08467ddebb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 STRUCT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.COL1->A NEW_KEY,\n AS_VALUE(INPUT.COL1->A) VV,\n COUNT(*) COUNT\nFROM INPUT INPUT\nGROUP BY INPUT.COL1->A\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "COL1->A" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "NEW_KEY" ], + "selectExpressions" : [ "AS_VALUE(COL1->A) AS VV", "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/spec.json new file mode 100644 index 000000000000..58efb127ba08 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/spec.json @@ -0,0 +1,172 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556344, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`A` STRING KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`A` STRING KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "struct in group-by and non aggregate function in select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : { + "a" : "lala", + "b" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : { + "a" : "lala", + "b" : 2 + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : { + "a" : "wonderland", + "b" : 3 + } + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : { + "a" : "lamb", + "b" : 4 + } + } + }, { + "topic" : "test_topic", + "key" : 4, + "value" : { + "col1" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "lala", + "value" : { + "VV" : "lala", + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "lala", + "value" : { + "VV" : "lala", + "COUNT" : 2 + } + }, { + "topic" : "OUTPUT", + "key" : "wonderland", + "value" : { + "VV" : "wonderland", + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "lamb", + "value" : { + "VV" : "lamb", + "COUNT" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 STRUCT) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1->a AS NEW_KEY, AS_VALUE(col1->a) as VV, COUNT(*) AS COUNT FROM INPUT GROUP BY col1->a;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` STRUCT<`A` STRING, `B` INTEGER>", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_struct_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556344/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_-_subscript_in_group-by_and_having/6.1.0_1597866556171/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/plan.json new file mode 100644 index 000000000000..edb067969ca4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/plan.json @@ -0,0 +1,186 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 MAP) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.COL1['foo'] NEW_KEY,\n COUNT(*) COUNT\nFROM INPUT INPUT\nGROUP BY INPUT.COL1['foo']\nHAVING (INPUT.COL1['foo'] = 'lala')\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "COL1['foo']" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "filterExpression" : "(COL1['foo'] = 'lala')" + }, + "keyColumnNames" : [ "NEW_KEY" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/spec.json new file mode 100644 index 000000000000..de6d0ccfa149 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/spec.json @@ -0,0 +1,150 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556171, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "subscript in group-by and having", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : { + "foo" : "lala" + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : { + "foo" : "kaka" + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : { + "alice" : "wonderland" + } + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : { + "mary" : "lamb" + } + } + }, { + "topic" : "test_topic", + "key" : 4, + "value" : { + "col1" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "lala", + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "kaka", + "value" : null + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'] HAVING col1['foo']='lala';" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/topology new file mode 100644 index 000000000000..b08ea0de8518 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_having/6.1.0_1597866556171/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_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/plan.json new file mode 100644 index 000000000000..dd0b4bdd0c7f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 MAP) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.COL1['foo'] NEW_KEY,\n AS_VALUE(INPUT.COL1['foo']) VV,\n COUNT(*) COUNT\nFROM INPUT INPUT\nGROUP BY INPUT.COL1['foo']\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "COL1['foo']" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "NEW_KEY" ], + "selectExpressions" : [ "AS_VALUE(COL1['foo']) AS VV", "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/spec.json new file mode 100644 index 000000000000..f3acfce3e4aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/spec.json @@ -0,0 +1,154 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556241, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "subscript in group-by and non aggregate function in select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : { + "foo" : "lala" + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : { + "foo" : "kaka" + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : { + "alice" : "wonderland" + } + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : { + "mary" : "lamb" + } + } + }, { + "topic" : "test_topic", + "key" : 4, + "value" : { + "col1" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "lala", + "value" : { + "VV" : "lala", + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "kaka", + "value" : { + "VV" : "kaka", + "COUNT" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, AS_VALUE(col1['foo']) as VV, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'];" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`NEW_KEY` STRING KEY, `VV` STRING, `COUNT` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_non_aggregate_function_in_select/6.1.0_1597866556241/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_-_subscript_in_group-by_and_select/6.1.0_1597866556081/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/plan.json new file mode 100644 index 000000000000..7c4df71ca275 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/plan.json @@ -0,0 +1,179 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, COL1 MAP) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='json');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n INPUT.COL1['foo'] NEW_KEY,\n COUNT(*) COUNT\nFROM INPUT INPUT\nGROUP BY INPUT.COL1['foo']\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "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" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` INTEGER KEY, `COL1` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "COL1 AS COL1", "ROWTIME AS ROWTIME" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "groupByExpressions" : [ "COL1['foo']" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "nonAggregateColumns" : [ "COL1", "ROWTIME" ], + "aggregationFunctions" : [ "COUNT(ROWTIME)" ] + }, + "keyColumnNames" : [ "NEW_KEY" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "false", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "true", + "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.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/spec.json new file mode 100644 index 000000000000..910e5ce26e9d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/spec.json @@ -0,0 +1,152 @@ +{ + "version" : "6.1.0", + "timestamp" : 1597866556081, + "path" : "query-validation-tests/group-by.json", + "schemas" : { + "CTAS_OUTPUT_0.Aggregate.GroupBy" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`KSQL_COL_0` STRING KEY, `COL1` MAP, `ROWTIME` BIGINT, `KSQL_AGG_VARIABLE_0` BIGINT", + "serdeOptions" : [ ] + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "subscript in group-by and select", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "col1" : { + "foo" : "lala" + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "col1" : { + "foo" : "kaka" + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "col1" : { + "alice" : "wonderland" + } + } + }, { + "topic" : "test_topic", + "key" : 3, + "value" : { + "col1" : { + "mary" : "lamb" + } + } + }, { + "topic" : "test_topic", + "key" : 4, + "value" : { + "col1" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "lala", + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "kaka", + "value" : { + "COUNT" : 1 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'];" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`NEW_KEY` STRING KEY, `COUNT` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` INTEGER KEY, `COL1` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/topology new file mode 100644 index 000000000000..99948fd4b5c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/group-by_-_subscript_in_group-by_and_select/6.1.0_1597866556081/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/query-validation-tests/group-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json index 4858256e74f6..0fa84bc081ff 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/group-by.json @@ -362,6 +362,174 @@ ] } }, + { + "name": "subscript in group-by and select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'];" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": {"foo" : "lala"}}}, + {"topic": "test_topic", "key": 1, "value": {"col1": {"foo" : "kaka"}}}, + {"topic": "test_topic", "key": 2, "value": {"col1": {"alice" : "wonderland"}}}, + {"topic": "test_topic", "key": 3, "value": {"col1": {"mary" : "lamb"}}}, + {"topic": "test_topic", "key": 4, "value": {"col1": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "lala", "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": "kaka", "value": {"COUNT": 1}} + ] + }, + { + "name": "subscript in group-by and having", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'] HAVING col1['foo']='lala';" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": {"foo" : "lala"}}}, + {"topic": "test_topic", "key": 1, "value": {"col1": {"foo" : "kaka"}}}, + {"topic": "test_topic", "key": 2, "value": {"col1": {"alice" : "wonderland"}}}, + {"topic": "test_topic", "key": 3, "value": {"col1": {"mary" : "lamb"}}}, + {"topic": "test_topic", "key": 4, "value": {"col1": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "lala", "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": "kaka", "value": null} + ] + }, + { + "name": "subscript in group-by and non aggregate function in select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1['foo'] AS NEW_KEY, AS_VALUE(col1['foo']) as VV, COUNT(*) AS COUNT FROM INPUT GROUP BY col1['foo'];" + + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": {"foo" : "lala"}}}, + {"topic": "test_topic", "key": 1, "value": {"col1": {"foo" : "kaka"}}}, + {"topic": "test_topic", "key": 2, "value": {"col1": {"alice" : "wonderland"}}}, + {"topic": "test_topic", "key": 3, "value": {"col1": {"mary" : "lamb"}}}, + {"topic": "test_topic", "key": 4, "value": {"col1": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "lala", "value": {"VV": "lala","COUNT": 1}}, + {"topic": "OUTPUT", "key": "kaka", "value": {"VV": "kaka","COUNT": 1}} + ] + }, + { + "name": "struct in group-by and non aggregate function in select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 STRUCT) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1->a AS NEW_KEY, AS_VALUE(col1->a) as VV, COUNT(*) AS COUNT FROM INPUT GROUP BY col1->a;" + + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": {"a" : "lala", "b": 1}}}, + {"topic": "test_topic", "key": 1, "value": {"col1": {"a" : "lala", "b": 2}}}, + {"topic": "test_topic", "key": 2, "value": {"col1": {"a" : "wonderland", "b": 3}}}, + {"topic": "test_topic", "key": 3, "value": {"col1": {"a" : "lamb", "b": 4}}}, + {"topic": "test_topic", "key": 4, "value": {"col1": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "lala", "value": {"VV": "lala","COUNT": 1}}, + {"topic": "OUTPUT", "key": "lala", "value": {"VV": "lala","COUNT": 2}}, + {"topic": "OUTPUT", "key": "wonderland", "value": {"VV": "wonderland","COUNT": 1}}, + {"topic": "OUTPUT", "key": "lamb", "value": {"VV": "lamb","COUNT": 1}} + ] + }, + { + "name": "function in group-by and nested function in select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, TRIM(COL3) AS G3, concat(initcap(col1), col2, trim(col3)) AS foo, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, TRIM(col3);" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": "smells", "col2": "like", "col3": "teen spirit"}}, + {"topic": "test_topic", "key": 1, "value": {"col1": "the", "col2": "man who", "col3": "stole the world"}}, + {"topic": "test_topic", "key": 2, "value": {"col1": "smells", "col2": "like", "col3": "spring"}}, + {"topic": "test_topic", "key": 3, "value": {"col1": "smells", "col2": "like", "col3": " teen spirit "}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "Smells|+|like|+|teen spirit", "value": {"FOO": "Smellsliketeen spirit", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "The|+|man who|+|stole the world", "value": {"FOO": "Theman whostole the world", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "Smells|+|like|+|spring", "value": {"FOO": "Smellslikespring","KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "Smells|+|like|+|teen spirit", "value": {"FOO": "Smellsliketeen spirit","KSQL_COL_0": 2}} + ] + }, + { + "name": "group by column in nested non-aggregate function in select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, COL3 AS G3, concat(initcap(col1), col2, trim(col3)) AS foo, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, col3;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": "smells", "col2": "like", "col3": "teen spirit"}}, + {"topic": "test_topic", "key": 1, "value": {"col1": "the", "col2": "man who", "col3": "stole the world"}}, + {"topic": "test_topic", "key": 2, "value": {"col1": "smells", "col2": "like", "col3": "spring"}}, + {"topic": "test_topic", "key": 3, "value": {"col1": "smells", "col2": "like", "col3": " teen spirit "}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "Smells|+|like|+|teen spirit", "value": {"FOO": "Smellsliketeen spirit", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "The|+|man who|+|stole the world", "value": {"FOO": "Theman whostole the world", "KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "Smells|+|like|+|spring", "value": {"FOO": "Smellslikespring","KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "Smells|+|like|+| teen spirit ", "value": {"FOO": "Smellsliketeen spirit","KSQL_COL_0": 1}} + ] + }, + { + "name": "function group by column used in non-aggregate function in having", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 VARCHAR, col2 VARCHAR, col3 VARCHAR) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT INITCAP(COL1) AS G1, COL2 AS G2, trim(COL3) AS G3, COUNT(*) FROM input GROUP BY INITCAP(col1), col2, trim(col3) HAVING substring(trim(col3),1,4) = 'teen';" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": "smells", "col2": "like", "col3": "teen spirit"}}, + {"topic": "test_topic", "key": 1, "value": {"col1": "the", "col2": "man who", "col3": "stole the world"}}, + {"topic": "test_topic", "key": 2, "value": {"col1": "smells", "col2": "like", "col3": "spring"}}, + {"topic": "test_topic", "key": 3, "value": {"col1": "smells", "col2": "like", "col3": " teen spirit "}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "Smells|+|like|+|teen spirit", "value": {"KSQL_COL_0": 1}}, + {"topic": "OUTPUT", "key": "The|+|man who|+|stole the world", "value": null}, + {"topic": "OUTPUT", "key": "Smells|+|like|+|spring", "value": null}, + {"topic": "OUTPUT", "key": "Smells|+|like|+|teen spirit", "value": {"KSQL_COL_0": 2}} + ] + }, + { + "name": "arithmetic in group by column used in non-aggregate function in select", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 INT, col2 INT) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1+col2 AS G1, AS_VALUE(col1+col2), COUNT(*) FROM input GROUP BY col1+col2;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": 1, "col2": 1}}, + {"topic": "test_topic", "key": 1, "value": {"col1": 2, "col2": 2}}, + {"topic": "test_topic", "key": 2, "value": {"col1": 3, "col2": 3}}, + {"topic": "test_topic", "key": 3, "value": {"col1": 4, "col2": 4}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 2, "value": {"KSQL_COL_0": 2, "KSQL_COL_1": 1}}, + {"topic": "OUTPUT", "key": 4, "value": {"KSQL_COL_0": 4, "KSQL_COL_1": 1}}, + {"topic": "OUTPUT", "key": 6, "value": {"KSQL_COL_0": 6, "KSQL_COL_1": 1}}, + {"topic": "OUTPUT", "key": 8, "value": {"KSQL_COL_0": 8, "KSQL_COL_1": 1}} + ] + }, + { + "name": "expressions used in non-aggregate function in select whose children are not part of group-by", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 MAP, col2 MAP) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1['foo']+col2['bar'] AS G1, AS_VALUE(col1['foo']+col2['bar']), COUNT(*) FROM input GROUP BY col1['foo']+col2['bar'];" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"col1": {"a" : 1}, "col2": {"b" : 1}}}, + {"topic": "test_topic", "key": 1, "value": {"col1": {"foo" : 1}, "col2": {"bar" : 1}}}, + {"topic": "test_topic", "key": 2, "value": {"col1": {"bar" : 1}, "col2": {"foo" : 1}}}, + {"topic": "test_topic", "key": 3, "value": {"col1": {"foo" : 1}, "col2": {"foo" : 1}}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 2, "value": {"KSQL_COL_0": 2, "KSQL_COL_1": 1}} + ] + }, { "name": "unknown function", "statements": [ @@ -511,6 +679,17 @@ "message": "Duplicate value columns found in schema: `COUNT` BIGINT" } }, + { + "name": "map used in non-aggregate function in select when group by uses subscript", + "statements": [ + "CREATE STREAM INPUT (id INT KEY, col1 MAP) WITH (kafka_topic='test_topic', value_format='json');", + "CREATE TABLE OUTPUT AS SELECT col1['foo'], AS_VALUE(col1) AS foo, COUNT(*) FROM input GROUP BY col1['foo'];" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Non-aggregate SELECT expression(s) not part of GROUP BY: AS_VALUE(COL1)\nEither add the column(s) to the GROUP BY or remove them from the SELECT." + } + }, { "name": "complex UDAF params", "statements": [ @@ -1231,7 +1410,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Column used in aggregate SELECT expression(s) outside of aggregate functions not part of GROUP BY: F1" + "message": "Non-aggregate SELECT expression(s) not part of GROUP BY: (F1 * SUM(F2))\nEither add the column(s) to the GROUP BY or remove them from the SELECT." } }, {