From 635aad75036f42d7a64007641aa514980af4372a Mon Sep 17 00:00:00 2001 From: Andy Coates Date: Wed, 4 Mar 2020 13:40:02 +0000 Subject: [PATCH 1/3] chore: partial support for arbitrarily named key columns Partial fix for: #3536 First part of supporting key column names other than `ROWKEY`. With this initial pass you can now name your key columns anything you want in your `CREATE TABLE` and `CREATE STREAM` statements, e.g. ```sql CREATE STREAM S (ID INT KEY, NAME STRING) WITH (...); ``` Any GROUP BY, PARTITION BY or JOIN on the key column results any created data source having a key column with a matching name, e.g. ```sql -- schema of T: ID INT KEY, COUNT BIGINT CREATE TABLE T AS SELECT COUNT() AS COUNT FROM S GROUP BY ID; ``` Pull and push queries work as expected and quoted identifiers work too. However, this functionality is not complete yet. Hence it is guarded by the `ksql.any.key.name.enabled` feature flag, which defaults to off. The following big ticket items are remaining: * PARTITION BY a single value column should result in a stream with the key column that matches the value column name. * GROUP BY a single value column should result in a table with the key column that matches the value column name. * JOIN on a single value column should result in a stream/table with the key column that matches the value column name. This additional work will be tracked under the same ticket, e.g. #3536 --- .../confluent/ksql/cli/console/Console.java | 5 +- .../io/confluent/ksql/util/KsqlConfig.java | 9 ++ .../io/confluent/ksql/analyzer/Analyzer.java | 13 +- .../ddl/commands/CreateSourceFactory.java | 30 ++-- .../ksql/engine/InsertValuesExecutor.java | 60 ++++--- .../ksql/planner/LogicalPlanner.java | 63 ++++++-- .../ksql/planner/plan/AggregateNode.java | 87 +++++----- .../ksql/planner/plan/DataSourceNode.java | 33 ++-- .../ksql/planner/plan/FilterNode.java | 16 +- .../ksql/planner/plan/FlatMapNode.java | 60 +++---- .../confluent/ksql/planner/plan/JoinNode.java | 19 +-- .../plan/KsqlStructuredDataOutputNode.java | 21 +++ .../ksql/planner/plan/OutputNode.java | 16 +- .../confluent/ksql/planner/plan/PlanNode.java | 33 ++-- .../ksql/planner/plan/ProjectNode.java | 35 ++-- .../ksql/planner/plan/RepartitionNode.java | 15 +- .../ksql/structured/SchemaKStream.java | 16 +- .../ksql/structured/SchemaKTable.java | 11 +- .../ddl/commands/CreateSourceCommand.java | 11 +- .../ksql/execution/util/StructKeyUtil.java | 19 +-- .../metastore/model/StructuredDataSource.java | 11 ++ .../server/execution/PullQueryExecutor.java | 149 ++++++++++-------- .../confluent/ksql/serde/GenericRowSerDe.java | 2 +- .../streams/GroupByParamsFactory.java | 30 ++-- .../ksql/execution/streams/SourceBuilder.java | 28 ++-- 25 files changed, 430 insertions(+), 362 deletions(-) diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index dba350ed959f..4bf9592badaf 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -85,6 +85,7 @@ import io.confluent.ksql.util.HandlerMaps.ClassHandlerMap1; import io.confluent.ksql.util.HandlerMaps.Handler1; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.SchemaUtil; import io.confluent.ksql.util.TabularRow; import java.io.Closeable; import java.io.File; @@ -454,11 +455,11 @@ private static String formatFieldType( final Optional windowType, final String keyField ) { - if (field.getName().equals("ROWTIME")) { + if (field.getName().equals(SchemaUtil.ROWTIME_NAME.text())) { return String.format("%-16s %s", field.getSchema().toTypeString(), "(system)"); } - if (field.getName().equals("ROWKEY")) { + if (field.getName().equals(SchemaUtil.ROWKEY_NAME.text())) { final String wt = windowType .map(v -> " (Window type: " + v + ")") .orElse(""); diff --git a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 94c2be70e704..fe19b433b3ac 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -262,6 +262,8 @@ public class KsqlConfig extends AbstractConfig { + "behavior, and instead throw an exception to ensure that no data is missed, set " + "ksql.timestamp.skip.invalid to true."; + public static final String KSQL_ANY_KEY_NAME_ENABLED = "ksql.any.key.name.enabled"; + private enum ConfigGeneration { LEGACY, CURRENT @@ -623,6 +625,13 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) { Importance.MEDIUM, KSQL_TIMESTAMP_THROW_ON_INVALID_DOC ) + .define( + KSQL_ANY_KEY_NAME_ENABLED, + Type.BOOLEAN, + false, + Importance.LOW, + "Feature flag for removing restriction on key names - WIP, do not enable." + ) .withClientSslSupport(); for (final CompatibilityBreakingConfigDef compatibilityBreakingConfigDef diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index fc635db7ba9d..7c04029dfa11 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -576,7 +576,7 @@ private void visitSelectStar(final AllColumns allColumns) { // See https://github.com/confluentinc/ksql/issues/3731 for more info final List valueColumns = persistent && !analysis.isJoin() ? schema.value() - : systemColumnsToTheFront(schema.withMetaAndKeyColsInValue(windowed).value()); + : orderColumns(schema.withMetaAndKeyColsInValue(windowed).value(), schema); for (final Column column : valueColumns) { @@ -596,12 +596,15 @@ private void visitSelectStar(final AllColumns allColumns) { } } - private List systemColumnsToTheFront(final List columns) { - // When doing a `select *` the system columns should be at the front of the column list + private List orderColumns( + final List columns, + final LogicalSchema schema + ) { + // When doing a `select *` system and key columns should be at the front of the column list // but are added at the back during processing for performance reasons. // Switch them around here: - final Map> partitioned = columns.stream() - .collect(Collectors.groupingBy(c -> SchemaUtil.isSystemColumn(c.name()))); + final Map> partitioned = columns.stream().collect(Collectors + .groupingBy(c -> SchemaUtil.isSystemColumn(c.name()) || schema.isKeyColumn(c.name()))); final List all = partitioned.get(true); all.addAll(partitioned.get(false)); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java index 8ce37cdd3421..d803b9eb60f0 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/CreateSourceFactory.java @@ -86,7 +86,7 @@ public CreateStreamCommand createStreamCommand( ) { final SourceName sourceName = statement.getName(); final KsqlTopic topic = buildTopic(statement.getProperties(), serviceContext); - final LogicalSchema schema = buildSchema(statement.getElements()); + final LogicalSchema schema = buildSchema(statement.getElements(), ksqlConfig); final Optional keyFieldName = buildKeyFieldName(statement, schema); final Optional timestampColumn = buildTimestampColumn( ksqlConfig, @@ -121,7 +121,7 @@ public CreateTableCommand createTableCommand( ) { final SourceName sourceName = statement.getName(); final KsqlTopic topic = buildTopic(statement.getProperties(), serviceContext); - final LogicalSchema schema = buildSchema(statement.getElements()); + final LogicalSchema schema = buildSchema(statement.getElements(), ksqlConfig); final Optional keyFieldName = buildKeyFieldName(statement, schema); final Optional timestampColumn = buildTimestampColumn( ksqlConfig, @@ -165,7 +165,10 @@ private static Optional buildKeyFieldName( } } - private static LogicalSchema buildSchema(final TableElements tableElements) { + private static LogicalSchema buildSchema( + final TableElements tableElements, + final KsqlConfig ksqlConfig + ) { if (Iterables.isEmpty(tableElements)) { throw new KsqlException("The statement does not define any columns."); } @@ -177,14 +180,21 @@ private static LogicalSchema buildSchema(final TableElements tableElements) { throw new KsqlException("'" + e.getName().text() + "' is a reserved column name."); } - if (e.getNamespace() == Namespace.KEY) { - if (!isRowKey) { - throw new KsqlException("'" + e.getName().text() + "' is an invalid KEY column name. " - + "KSQL currently only supports KEY columns named ROWKEY."); + if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ANY_KEY_NAME_ENABLED)) { + if (isRowKey && e.getNamespace() != Namespace.KEY) { + throw new KsqlException("'" + e.getName().text() + "' is a reserved column name. " + + "It can only be used for KEY columns."); + } + } else { + if (e.getNamespace() == Namespace.KEY) { + if (!isRowKey) { + throw new KsqlException("'" + e.getName().text() + "' is an invalid KEY column name. " + + "KSQL currently only supports KEY columns named ROWKEY."); + } + } else if (isRowKey) { + throw new KsqlException("'" + e.getName().text() + "' is a reserved column name. " + + "It can only be used for KEY columns."); } - } else if (isRowKey) { - throw new KsqlException("'" + e.getName().text() + "' is a reserved column name. " - + "It can only be used for KEY columns."); } }); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index 33455079ed70..1329d9bea60e 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -37,7 +37,6 @@ import io.confluent.ksql.rest.SessionProperties; import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; -import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.ksql.SchemaConverters; @@ -61,7 +60,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.function.LongSupplier; @@ -142,6 +140,7 @@ private InsertValuesExecutor( this.valueSerdeFactory = Objects.requireNonNull(valueSerdeFactory, "valueSerdeFactory"); } + @SuppressWarnings("unused") // Part of required API. public void execute( final ConfiguredStatement statement, final SessionProperties sessionProperties, @@ -175,7 +174,7 @@ public void execute( } } - private DataSource getDataSource( + private static DataSource getDataSource( final KsqlConfig ksqlConfig, final MetaStore metaStore, final InsertValues insertValues @@ -266,11 +265,23 @@ private RowData extractRow( final Map values = resolveValues( insertValues, columns, schema, functionRegistry, config); - handleExplicitKeyField(values, dataSource.getKeyField()); + handleExplicitKeyField( + values, + dataSource.getKeyField(), + Iterables.getOnlyElement(dataSource.getSchema().key()) + ); + + if (dataSource.getDataSourceType() == DataSourceType.KTABLE) { + final String noValue = dataSource.getSchema().key().stream() + .map(Column::name) + .filter(colName -> !values.containsKey(colName)) + .map(ColumnName::text) + .collect(Collectors.joining(", ")); - if (dataSource.getDataSourceType() == DataSourceType.KTABLE - && values.get(SchemaUtil.ROWKEY_NAME) == null) { - throw new KsqlException("Value for ROWKEY is required for tables"); + if (!noValue.isEmpty()) { + throw new KsqlException("Value for primary key column(s) " + + noValue + " is required for tables"); + } } final long ts = (long) values.getOrDefault(SchemaUtil.ROWTIME_NAME, clock.getAsLong()); @@ -357,26 +368,29 @@ private static Map resolveValues( private static void handleExplicitKeyField( final Map values, - final KeyField keyField + final KeyField keyField, + final Column keyColumn ) { - final Optional keyFieldName = keyField.ref(); - if (keyFieldName.isPresent()) { - final ColumnName key = keyFieldName.get(); - final Object keyValue = values.get(key); - final Object rowKeyValue = values.get(SchemaUtil.ROWKEY_NAME); - - if (keyValue != null ^ rowKeyValue != null) { - if (keyValue == null) { - values.put(key, rowKeyValue); + // key column: the key column in the source's schema. + // key field: the column identified in the WITH clause as being an alias to the key column. + + keyField.ref().ifPresent(keyFieldName -> { + final ColumnName keyColumnName = keyColumn.name(); + final Object keyFieldValue = values.get(keyFieldName); + final Object keyColumnValue = values.get(keyColumnName); + + if (keyFieldValue != null ^ keyColumnValue != null) { + if (keyFieldValue == null) { + values.put(keyFieldName, keyColumnValue); } else { - values.put(SchemaUtil.ROWKEY_NAME, keyValue); + values.put(keyColumnName, keyFieldValue); } - } else if (keyValue != null && !Objects.equals(keyValue, rowKeyValue)) { - throw new KsqlException(String.format( - "Expected ROWKEY and %s to match but got %s and %s respectively.", - key.toString(FormatOptions.noEscape()), rowKeyValue, keyValue)); + } else if (keyFieldValue != null && !Objects.equals(keyFieldValue, keyColumnValue)) { + throw new KsqlException( + "Expected " + keyColumnName.text() + " and " + keyFieldName.text() + " to match " + + "but got " + keyColumnValue + " and " + keyFieldValue + " respectively."); } - } + }); } private static SqlType columnType(final ColumnName column, final LogicalSchema schema) { diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java index 69b27fbbee0e..c3324e8d43d2 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/LogicalPlanner.java @@ -51,6 +51,7 @@ import io.confluent.ksql.planner.plan.ProjectNode; import io.confluent.ksql.planner.plan.RepartitionNode; import io.confluent.ksql.schema.ksql.Column; +import io.confluent.ksql.schema.ksql.Column.Namespace; import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.LogicalSchema.Builder; @@ -173,8 +174,11 @@ private AggregateNode buildAggregateNode(final PlanNode sourcePlanNode) { ? groupByExps.get(0) : null; - final Optional keyFieldName = getSelectAliasMatching((expression, alias) -> - expression.equals(groupBy) && !SchemaUtil.isSystemColumn(alias), + final Optional keyFieldName = getSelectAliasMatching( + (expression, alias) -> + expression.equals(groupBy) + && !SchemaUtil.isSystemColumn(alias) + && !schema.isKeyColumn(alias), sourcePlanNode.getSelectExpressions()); return new AggregateNode( @@ -389,9 +393,7 @@ private LogicalSchema buildProjectionSchema( final Builder builder = LogicalSchema.builder() .withRowTime(); - final List keyColumns = schema.key(); - - builder.keyColumns(keyColumns); + builder.keyColumns(schema.key()); for (int i = 0; i < projection.size(); i++) { final SelectExpression select = projection.get(i); @@ -409,26 +411,36 @@ private LogicalSchema buildAggregateSchema( final PlanNode sourcePlanNode, final List groupByExps ) { + final LogicalSchema sourceSchema = sourcePlanNode.getSchema(); + + final ColumnName keyName; final SqlType keyType; if (groupByExps.size() != 1) { + keyName = SchemaUtil.ROWKEY_NAME; keyType = SqlTypes.STRING; } else { + final Expression expression = groupByExps.get(0); + + keyName = exactlyMatchesKeyColumns(expression, sourceSchema) + ? ((ColumnReferenceExp) expression).getColumnName() + : SchemaUtil.ROWKEY_NAME; + final ExpressionTypeManager typeManager = - new ExpressionTypeManager(sourcePlanNode.getSchema(), functionRegistry); + new ExpressionTypeManager(sourceSchema, functionRegistry); - keyType = typeManager.getExpressionSqlType(groupByExps.get(0)); + keyType = typeManager.getExpressionSqlType(expression); } - final LogicalSchema sourceSchema = buildProjectionSchema( - sourcePlanNode.getSchema() + final LogicalSchema projectionSchema = buildProjectionSchema( + sourceSchema .withMetaAndKeyColsInValue(analysis.getWindowExpression().isPresent()), sourcePlanNode.getSelectExpressions() ); return LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) - .valueColumns(sourceSchema.value()) + .keyColumn(keyName, keyType) + .valueColumns(projectionSchema.value()) .build(); } @@ -438,6 +450,11 @@ private LogicalSchema buildRepartitionedSchema( ) { final LogicalSchema sourceSchema = sourceNode.getSchema(); + if (exactlyMatchesKeyColumns(partitionBy, sourceSchema)) { + // No-op: + return sourceSchema; + } + final ExpressionTypeManager typeManager = new ExpressionTypeManager(sourceSchema, functionRegistry); @@ -450,6 +467,30 @@ private LogicalSchema buildRepartitionedSchema( .build(); } + private static boolean exactlyMatchesKeyColumns( + final Expression expression, + final LogicalSchema schema + ) { + if (schema.key().size() != 1) { + // Currently only support single key column: + return false; + } + + if (!(expression instanceof ColumnReferenceExp)) { + // Anything not a column ref can't be a match: + return false; + } + + final ColumnName columnName = ((ColumnReferenceExp) expression).getColumnName(); + + final Namespace ns = schema + .findColumn(columnName) + .map(Column::namespace) + .orElse(Namespace.VALUE); + + return ns == Namespace.KEY; + } + private static List selectWithPrependAlias( final SourceName alias, final LogicalSchema schema diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java index a95676968b44..6a821b847df1 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java @@ -65,14 +65,12 @@ public class AggregateNode extends PlanNode { private static final String PROJECT_OP_NAME = "Project"; private final PlanNode source; - private final LogicalSchema schema; private final KeyField keyField; private final ImmutableList groupByExpressions; private final Optional windowExpression; private final ImmutableList aggregateFunctionArguments; private final ImmutableList functionList; private final ImmutableList requiredColumns; - private final ImmutableList finalSelectExpressions; private final Expression havingExpressions; // CHECKSTYLE_RULES.OFF: ParameterNumberCheck @@ -90,10 +88,14 @@ public AggregateNode( final Expression havingExpressions ) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck - super(id, DataSourceType.KTABLE); + super( + id, + DataSourceType.KTABLE, + schema, + buildSelectExpressions(schema, finalSelectExpressions) + ); this.source = requireNonNull(source, "source"); - this.schema = requireNonNull(schema, "schema"); this.groupByExpressions = ImmutableList .copyOf(requireNonNull(groupByExpressions, "groupByExpressions")); this.windowExpression = requireNonNull(windowExpression, "windowExpression"); @@ -103,18 +105,11 @@ public AggregateNode( .copyOf(requireNonNull(functionList, "functionList")); this.requiredColumns = ImmutableList .copyOf(requireNonNull(requiredColumns, "requiredColumns")); - this.finalSelectExpressions = ImmutableList - .copyOf(requireNonNull(finalSelectExpressions, "finalSelectExpressions")); this.havingExpressions = havingExpressions; this.keyField = KeyField.of(requireNonNull(keyFieldName, "keyFieldName")) .validateKeyExistsIn(schema); } - @Override - public LogicalSchema getSchema() { - return schema; - } - @Override public KeyField getKeyField() { return keyField; @@ -145,30 +140,6 @@ public List getRequiredColumns() { return requiredColumns; } - private List getFinalSelectExpressions() { - final List finalSelectExpressionList = new ArrayList<>(); - if (finalSelectExpressions.size() != schema.value().size()) { - throw new RuntimeException( - "Incompatible aggregate schema, field count must match, " - + "selected field count:" - + finalSelectExpressions.size() - + " schema field count:" - + schema.value().size()); - } - for (int i = 0; i < finalSelectExpressions.size(); i++) { - finalSelectExpressionList.add(SelectExpression.of( - schema.value().get(i).name(), - finalSelectExpressions.get(i) - )); - } - return finalSelectExpressionList; - } - - @Override - public List getSelectExpressions() { - return Collections.emptyList(); - } - @Override public R accept(final PlanVisitor visitor, final C context) { return visitor.visitAggregate(this, context); @@ -182,6 +153,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { // Pre aggregate computations final InternalSchema internalSchema = new InternalSchema( + getSchema(), requiredColumns, aggregateFunctionArguments ); @@ -245,7 +217,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { } final List finalSelects = internalSchema - .updateFinalSelectExpressions(getFinalSelectExpressions()); + .updateFinalSelectExpressions(getSelectExpressions()); return aggregated.select( finalSelects, @@ -258,14 +230,43 @@ protected int getPartitions(final KafkaTopicClient kafkaTopicClient) { return source.getPartitions(kafkaTopicClient); } + private static List buildSelectExpressions( + final LogicalSchema schema, + final List finalSelectExpressions + ) { + final List finalSelectExpressionList = new ArrayList<>(); + if (finalSelectExpressions.size() != schema.value().size()) { + throw new RuntimeException( + "Incompatible aggregate schema, field count must match, " + + "selected field count:" + + finalSelectExpressions.size() + + " schema field count:" + + schema.value().size()); + } + for (int i = 0; i < finalSelectExpressions.size(); i++) { + finalSelectExpressionList.add(SelectExpression.of( + schema.value().get(i).name(), + finalSelectExpressions.get(i) + )); + } + return finalSelectExpressionList; + } + private static class InternalSchema { + private final Optional singleKeyColumn; private final List aggArgExpansions = new ArrayList<>(); private final Map expressionToInternalColumnName = new HashMap<>(); InternalSchema( + final LogicalSchema schema, final List requiredColumns, - final List aggregateFunctionArguments) { + final List aggregateFunctionArguments + ) { + this.singleKeyColumn = schema.key().size() == 1 + ? Optional.of(schema.key().get(0).name()) + : Optional.empty(); + final Set seen = new HashSet<>(); collectAggregateArgExpressions(requiredColumns, seen); collectAggregateArgExpressions(aggregateFunctionArguments, seen); @@ -297,19 +298,15 @@ List resolveGroupByExpressions( final boolean specialRowTimeHandling = !(aggregateArgExpanded instanceof SchemaKTable); final Function mapper = e -> { - final boolean rowKey = e instanceof UnqualifiedColumnReferenceExp - && ((UnqualifiedColumnReferenceExp) e).getColumnName().equals( - SchemaUtil.ROWKEY_NAME); + final boolean rowKey = singleKeyColumn.isPresent() + && e instanceof UnqualifiedColumnReferenceExp + && ((UnqualifiedColumnReferenceExp) e).getColumnName().equals(singleKeyColumn.get()); if (!rowKey || !specialRowTimeHandling) { return resolveToInternal(e); } - final UnqualifiedColumnReferenceExp nameRef = (UnqualifiedColumnReferenceExp) e; - return new UnqualifiedColumnReferenceExp( - nameRef.getLocation(), - nameRef.getColumnName() - ); + return e; }; return expressionList.stream() diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java index 48b73e88f8c8..c14fc192d392 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java @@ -40,10 +40,8 @@ public class DataSourceNode extends PlanNode { private final DataSource dataSource; private final SourceName alias; - private final LogicalSchema schema; private final KeyField keyField; private final SchemaKStreamFactory schemaKStreamFactory; - private final ImmutableList selectExpressions; public DataSourceNode( final PlanNodeId id, @@ -61,29 +59,16 @@ public DataSourceNode( final List selectExpressions, final SchemaKStreamFactory schemaKStreamFactory ) { - super(id, dataSource.getDataSourceType()); + super(id, dataSource.getDataSourceType(), buildSchema(dataSource), selectExpressions); this.dataSource = requireNonNull(dataSource, "dataSource"); this.alias = requireNonNull(alias, "alias"); - this.selectExpressions = - ImmutableList.copyOf(requireNonNull(selectExpressions, "selectExpressions")); - - // DataSourceNode copies implicit and key fields into the value schema - // It users a KS valueMapper to add the key fields - // and a KS transformValues to add the implicit fields - this.schema = dataSource.getSchema() - .withMetaAndKeyColsInValue(dataSource.getKsqlTopic().getKeyFormat().isWindowed()); this.keyField = dataSource.getKeyField() - .validateKeyExistsIn(schema); + .validateKeyExistsIn(getSchema()); this.schemaKStreamFactory = requireNonNull(schemaKStreamFactory, "schemaKStreamFactory"); } - @Override - public LogicalSchema getSchema() { - return schema; - } - @Override public KeyField getKeyField() { return keyField; @@ -115,11 +100,6 @@ public List getSources() { return ImmutableList.of(); } - @Override - public List getSelectExpressions() { - return selectExpressions; - } - @Override public R accept(final PlanVisitor visitor, final C context) { return visitor.visitDataSourceNode(this, context); @@ -136,8 +116,17 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { ); } + private static LogicalSchema buildSchema(final DataSource dataSource) { + // DataSourceNode copies implicit and key fields into the value schema + // It users a KS valueMapper to add the key fields + // and a KS transformValues to add the implicit fields + return dataSource.getSchema() + .withMetaAndKeyColsInValue(dataSource.getKsqlTopic().getKeyFormat().isWindowed()); + } + @Immutable interface SchemaKStreamFactory { + SchemaKStream create( KsqlQueryBuilder builder, DataSource dataSource, diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java index a648904cc156..7971c487eb80 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FilterNode.java @@ -20,9 +20,7 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext.Stacker; import io.confluent.ksql.execution.expression.tree.Expression; -import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.metastore.model.KeyField; -import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.services.KafkaTopicClient; import io.confluent.ksql.structured.SchemaKStream; import java.util.List; @@ -33,29 +31,22 @@ public class FilterNode extends PlanNode { private final PlanNode source; private final Expression predicate; - private final ImmutableList selectExpressions; public FilterNode( final PlanNodeId id, final PlanNode source, final Expression predicate ) { - super(id, source.getNodeOutputType()); + super(id, source.getNodeOutputType(), source.getSchema(), source.getSelectExpressions()); this.source = Objects.requireNonNull(source, "source"); this.predicate = Objects.requireNonNull(predicate, "predicate"); - this.selectExpressions = ImmutableList.copyOf(source.getSelectExpressions()); } public Expression getPredicate() { return predicate; } - @Override - public LogicalSchema getSchema() { - return source.getSchema(); - } - @Override public KeyField getKeyField() { return source.getKeyField(); @@ -66,11 +57,6 @@ public List getSources() { return ImmutableList.of(source); } - @Override - public List getSelectExpressions() { - return selectExpressions; - } - public PlanNode getSource() { return source; } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java index 3a849e395448..952557380266 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/FlatMapNode.java @@ -48,10 +48,7 @@ public class FlatMapNode extends PlanNode { private final PlanNode source; - private final LogicalSchema outputSchema; - private final ImmutableList finalSelectExpressions; - private final ImmutableAnalysis analysis; - private final FunctionRegistry functionRegistry; + private final ImmutableList tableFunctions; public FlatMapNode( final PlanNodeId id, @@ -59,21 +56,14 @@ public FlatMapNode( final FunctionRegistry functionRegistry, final ImmutableAnalysis analysis ) { - super(id, source.getNodeOutputType()); - this.source = Objects.requireNonNull(source, "source"); - this.analysis = Objects.requireNonNull(analysis); - this.functionRegistry = functionRegistry; - this.finalSelectExpressions = buildFinalSelectExpressions(); - outputSchema = StreamFlatMapBuilder.buildSchema( - source.getSchema(), - analysis.getTableFunctions(), - functionRegistry + super( + id, + source.getNodeOutputType(), + buildSchema(source, functionRegistry, analysis), + buildFinalSelectExpressions(functionRegistry, analysis) ); - } - - @Override - public LogicalSchema getSchema() { - return outputSchema; + this.source = Objects.requireNonNull(source, "source"); + this.tableFunctions = ImmutableList.copyOf(analysis.getTableFunctions()); } @Override @@ -90,11 +80,6 @@ public PlanNode getSource() { return source; } - @Override - public List getSelectExpressions() { - return finalSelectExpressions; - } - @Override public R accept(final PlanVisitor visitor, final C context) { return visitor.visitFlatMap(this, context); @@ -111,14 +96,17 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { final QueryContext.Stacker contextStacker = builder.buildNodeContext(getId().toString()); return getSource().buildStream(builder).flatMap( - analysis.getTableFunctions(), + tableFunctions, contextStacker ); } - private ImmutableList buildFinalSelectExpressions() { + private static ImmutableList buildFinalSelectExpressions( + final FunctionRegistry functionRegistry, + final ImmutableAnalysis analysis + ) { final TableFunctionExpressionRewriter tableFunctionExpressionRewriter = - new TableFunctionExpressionRewriter(); + new TableFunctionExpressionRewriter(functionRegistry); final ImmutableList.Builder selectExpressions = ImmutableList.builder(); for (final SelectExpression select : analysis.getSelectExpressions()) { @@ -133,13 +121,17 @@ private ImmutableList buildFinalSelectExpressions() { return selectExpressions.build(); } - private class TableFunctionExpressionRewriter + private static class TableFunctionExpressionRewriter extends VisitParentExpressionVisitor, Context> { + private final FunctionRegistry functionRegistry; private int variableIndex = 0; - TableFunctionExpressionRewriter() { + TableFunctionExpressionRewriter( + final FunctionRegistry functionRegistry + ) { super(Optional.empty()); + this.functionRegistry = Objects.requireNonNull(functionRegistry, "functionRegistry"); } @Override @@ -163,4 +155,16 @@ public Optional visitFunctionCall( } } } + + private static LogicalSchema buildSchema( + final PlanNode source, + final FunctionRegistry functionRegistry, + final ImmutableAnalysis analysis + ) { + return StreamFlatMapBuilder.buildSchema( + source.getSchema(), + analysis.getTableFunctions(), + functionRegistry + ); + } } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java index f7862875f871..b447efceb472 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/JoinNode.java @@ -15,7 +15,6 @@ package io.confluent.ksql.planner.plan; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; @@ -49,10 +48,8 @@ public enum JoinType { private final JoinType joinType; private final PlanNode left; private final PlanNode right; - private final LogicalSchema schema; private final KeyField keyField; private final Optional withinExpression; - private final ImmutableList selectExpressions; public JoinNode( final PlanNodeId id, @@ -62,24 +59,15 @@ public JoinNode( final PlanNode right, final Optional withinExpression ) { - super(id, calculateSinkType(left, right)); + super(id, calculateSinkType(left, right), buildJoinSchema(left, right), selectExpressions); this.joinType = Objects.requireNonNull(joinType, "joinType"); this.left = Objects.requireNonNull(left, "left"); this.right = Objects.requireNonNull(right, "right"); this.withinExpression = Objects.requireNonNull(withinExpression, "withinExpression"); - this.selectExpressions = ImmutableList - .copyOf(Objects.requireNonNull(selectExpressions, "selectExpressions")); this.keyField = joinType == JoinType.OUTER ? KeyField.none() // Both source key columns can be null, hence neither can be the keyField : left.getKeyField(); - - this.schema = buildJoinSchema(left, right); - } - - @Override - public LogicalSchema getSchema() { - return schema; } @Override @@ -97,11 +85,6 @@ public R accept(final PlanVisitor visitor, final C context) { return visitor.visitJoin(this, context); } - @Override - public List getSelectExpressions() { - return selectExpressions; - } - public PlanNode getLeft() { return left; } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java index 392d3096d8eb..a8eca3f45232 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java @@ -22,17 +22,21 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; +import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.metastore.model.KeyField; +import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.query.id.QueryIdGenerator; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.structured.SchemaKStream; +import io.confluent.ksql.util.KsqlException; import java.util.Optional; import java.util.OptionalInt; import java.util.Set; +import java.util.stream.Collectors; public class KsqlStructuredDataOutputNode extends OutputNode { @@ -73,6 +77,8 @@ public KsqlStructuredDataOutputNode( this.ksqlTopic = requireNonNull(ksqlTopic, "ksqlTopic"); this.doCreateInto = doCreateInto; this.intoSourceName = requireNonNull(intoSourceName, "intoSourceName"); + + validate(); } public boolean isDoCreateInto() { @@ -123,4 +129,19 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { getTimestampColumn() ); } + + private void validate() { + final LogicalSchema schema = getSchema(); + + final String duplicates = getSelectExpressions().stream() + .map(SelectExpression::getAlias) + .filter(schema::isKeyColumn) + .map(ColumnName::toString) + .collect(Collectors.joining(", ")); + + if (!duplicates.isEmpty()) { + throw new KsqlException("Value column name(s) " + duplicates + + " clashes with key column name(s). Please remove or alias the column(s)."); + } + } } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/OutputNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/OutputNode.java index 51e066ec5c54..01b1a4d28080 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/OutputNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/OutputNode.java @@ -19,13 +19,11 @@ import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.Immutable; -import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.query.id.QueryIdGenerator; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.services.KafkaTopicClient; -import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.OptionalInt; @@ -36,7 +34,6 @@ public abstract class OutputNode extends PlanNode { private final PlanNode source; - private final LogicalSchema schema; private final OptionalInt limit; private final Optional timestampColumn; @@ -47,20 +44,14 @@ protected OutputNode( final OptionalInt limit, final Optional timestampColumn ) { - super(id, source.getNodeOutputType()); + super(id, source.getNodeOutputType(), schema, source.getSelectExpressions()); this.source = requireNonNull(source, "source"); - this.schema = requireNonNull(schema, "schema"); this.limit = requireNonNull(limit, "limit"); this.timestampColumn = requireNonNull(timestampColumn, "timestampColumn"); } - @Override - public LogicalSchema getSchema() { - return schema; - } - @Override public List getSources() { return ImmutableList.of(source); @@ -79,11 +70,6 @@ protected int getPartitions(final KafkaTopicClient kafkaTopicClient) { return source.getPartitions(kafkaTopicClient); } - @Override - public List getSelectExpressions() { - return Collections.emptyList(); - } - @Override public R accept(final PlanVisitor visitor, final C context) { return visitor.visitOutput(this, context); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java index 81bd2c4abd38..eb934467d919 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/PlanNode.java @@ -17,6 +17,7 @@ import static java.util.Objects.requireNonNull; +import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.plan.SelectExpression; @@ -32,29 +33,41 @@ public abstract class PlanNode { private final PlanNodeId id; private final DataSourceType nodeOutputType; + private final LogicalSchema schema; + private final ImmutableList selectExpressions; - protected PlanNode(final PlanNodeId id, final DataSourceType nodeOutputType) { - requireNonNull(id, "id is null"); - requireNonNull(nodeOutputType, "nodeOutputType is null"); - this.id = id; - this.nodeOutputType = nodeOutputType; + protected PlanNode( + final PlanNodeId id, + final DataSourceType nodeOutputType, + final LogicalSchema schema, + final List selectExpressions + ) { + this.id = requireNonNull(id, "id"); + this.nodeOutputType = requireNonNull(nodeOutputType, "nodeOutputType"); + this.schema = requireNonNull(schema, "schema"); + this.selectExpressions = ImmutableList + .copyOf(requireNonNull(selectExpressions, "projectExpressions")); } - public PlanNodeId getId() { + public final PlanNodeId getId() { return id; } - public DataSourceType getNodeOutputType() { + public final DataSourceType getNodeOutputType() { return nodeOutputType; } - public abstract LogicalSchema getSchema(); + public final LogicalSchema getSchema() { + return schema; + } + + public final List getSelectExpressions() { + return selectExpressions; + } public abstract KeyField getKeyField(); public abstract List getSources(); - - public abstract List getSelectExpressions(); public R accept(final PlanVisitor visitor, final C context) { return visitor.visitPlan(this, context); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java index 619d84cd4121..32df132305fb 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/ProjectNode.java @@ -29,15 +29,12 @@ import io.confluent.ksql.structured.SchemaKStream; import io.confluent.ksql.util.KsqlException; import java.util.List; -import java.util.Objects; import java.util.Optional; @Immutable public class ProjectNode extends PlanNode { private final PlanNode source; - private final LogicalSchema schema; - private final ImmutableList projectExpressions; private final KeyField keyField; public ProjectNode( @@ -47,21 +44,12 @@ public ProjectNode( final LogicalSchema schema, final Optional keyFieldName ) { - super(id, source.getNodeOutputType()); + super(id, source.getNodeOutputType(), schema, projectExpressions); this.source = requireNonNull(source, "source"); - this.schema = requireNonNull(schema, "schema"); - this.projectExpressions = ImmutableList.copyOf( - Objects.requireNonNull(projectExpressions, "projectExpressions") - ); this.keyField = KeyField.of(requireNonNull(keyFieldName, "keyFieldName")) .validateKeyExistsIn(schema); - if (schema.value().size() != projectExpressions.size()) { - throw new KsqlException("Error in projection. Schema fields and expression list are not " - + "compatible."); - } - validate(); } @@ -74,11 +62,6 @@ public PlanNode getSource() { return source; } - @Override - public LogicalSchema getSchema() { - return schema; - } - @Override protected int getPartitions(final KafkaTopicClient kafkaTopicClient) { return source.getPartitions(kafkaTopicClient); @@ -89,11 +72,6 @@ public KeyField getKeyField() { return keyField; } - @Override - public List getSelectExpressions() { - return projectExpressions; - } - @Override public R accept(final PlanVisitor visitor, final C context) { return visitor.visitProject(this, context); @@ -110,9 +88,14 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { } private void validate() { - for (int i = 0; i < projectExpressions.size(); i++) { - final Column column = schema.value().get(i); - final SelectExpression selectExpression = projectExpressions.get(i); + if (getSchema().value().size() != getSelectExpressions().size()) { + throw new KsqlException("Error in projection. Schema fields and expression list are not " + + "compatible."); + } + + for (int i = 0; i < getSelectExpressions().size(); i++) { + final Column column = getSchema().value().get(i); + final SelectExpression selectExpression = getSelectExpressions().get(i); if (!column.name().equals(selectExpression.getAlias())) { throw new IllegalArgumentException("Mismatch between schema and selects"); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java index ba3bea90020d..8a08784226c7 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/RepartitionNode.java @@ -22,7 +22,6 @@ import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.expression.tree.Expression; -import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.services.KafkaTopicClient; @@ -35,7 +34,6 @@ public class RepartitionNode extends PlanNode { private final PlanNode source; private final Expression partitionBy; private final KeyField keyField; - private final LogicalSchema schema; public RepartitionNode( final PlanNodeId id, @@ -44,16 +42,10 @@ public RepartitionNode( final Expression partitionBy, final KeyField keyField ) { - super(id, source.getNodeOutputType()); + super(id, source.getNodeOutputType(), schema, source.getSelectExpressions()); this.source = requireNonNull(source, "source"); this.partitionBy = requireNonNull(partitionBy, "partitionBy"); this.keyField = requireNonNull(keyField, "keyField"); - this.schema = requireNonNull(schema, "schema"); - } - - @Override - public LogicalSchema getSchema() { - return schema; } @Override @@ -66,11 +58,6 @@ public List getSources() { return ImmutableList.of(source); } - @Override - public List getSelectExpressions() { - return source.getSelectExpressions(); - } - @Override protected int getPartitions(final KafkaTopicClient kafkaTopicClient) { return source.getPartitions(kafkaTopicClient); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java index aa36d73ea6e6..c4ccc39b0e08 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java @@ -358,7 +358,7 @@ private KeyField getNewKeyField(final Expression expression) { } final ColumnName columnName = ((UnqualifiedColumnReferenceExp) expression).getColumnName(); - final KeyField newKeyField = isRowKey(columnName) ? keyField : KeyField.of(columnName); + final KeyField newKeyField = isKeyColumn(columnName) ? keyField : KeyField.of(columnName); return getSchema().isMetaColumn(columnName) ? KeyField.none() : newKeyField; } @@ -381,12 +381,12 @@ protected boolean repartitionNotNeeded(final Expression expression) { .map(kf -> kf.name().equals(proposedKey.name())) .orElse(false); - return namesMatch || isRowKey(columnName); + return namesMatch || isKeyColumn(columnName); } - private boolean isRowKey(final ColumnName fieldName) { - // until we support structured keys, there will never be any key column other - // than "ROWKEY" - furthermore, that key column is always prefixed at this point + private boolean isKeyColumn(final ColumnName fieldName) { + // until we support structured keys, there will only be a single key column + // - furthermore, that key column is always prefixed at this point // unless it is a join, in which case every other source field is prefixed return fieldName.equals(schema.key().get(0).name()); } @@ -404,12 +404,16 @@ private boolean rekeyRequired(final List groupByExpressions) { return true; } + if (schema.key().size() != 1) { + return true; + } + final ColumnName groupByField = fieldNameFromExpression(groupByExpressions.get(0)); if (groupByField == null) { return true; } - if (groupByField.equals(SchemaUtil.ROWKEY_NAME)) { + if (groupByField.equals(schema.key().get(0).name())) { return false; } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java index 3f9b3c5fa51d..18ed558bbf82 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java @@ -15,6 +15,7 @@ package io.confluent.ksql.structured; +import com.google.common.collect.Iterables; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.context.QueryContext.Stacker; @@ -140,15 +141,17 @@ public SchemaKTable select( @SuppressWarnings("unchecked") @Override - public SchemaKStream selectKey(final Expression keyExpression, - final Stacker contextStacker) { + public SchemaKStream selectKey( + final Expression keyExpression, + final Stacker contextStacker + ) { if (repartitionNotNeeded(keyExpression)) { return (SchemaKStream) this; } throw new UnsupportedOperationException("Cannot repartition a TABLE source. " - + "If this is a join, make sure that the criteria uses the TABLE key " - + this.keyField.ref().map(ColumnName::toString).orElse("ROWKEY") + " instead of " + + "If this is a join, make sure that the criteria uses the TABLE's key column " + + Iterables.getOnlyElement(schema.key()).name().text() + " instead of " + keyExpression); } diff --git a/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java b/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java index 2282fdce9559..4decad802764 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommand.java @@ -20,7 +20,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.schema.ksql.Column; -import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.serde.WindowInfo; @@ -108,16 +107,16 @@ private static void validate(final LogicalSchema schema, final Optional implements DataSource { @@ -66,6 +69,14 @@ abstract class StructuredDataSource implements DataSource { if (schema.valueContainsAny(SchemaUtil.systemColumnNames())) { throw new IllegalArgumentException("Schema contains system columns in value schema"); } + + final Set keyNames = schema.key().stream() + .map(Column::name) + .collect(Collectors.toSet()); + + if (schema.valueContainsAny(keyNames)) { + throw new IllegalArgumentException("Schema contains duplicate column names"); + } } @Override diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java index a193a31c850c..72e9cbb1eb86 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java @@ -62,6 +62,7 @@ import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.tree.AllColumns; import io.confluent.ksql.parser.tree.Query; @@ -76,6 +77,7 @@ import io.confluent.ksql.rest.entity.TableRowsEntity; import io.confluent.ksql.rest.entity.TableRowsEntityFactory; import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; import io.confluent.ksql.schema.ksql.FormatOptions; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -104,6 +106,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,6 +140,7 @@ public PullQueryExecutor( Objects.requireNonNull(routingFilterFactory, "routingFilterFactory"); } + @SuppressWarnings("unused") // Needs to match validator API. public static void validate( final ConfiguredStatement statement, final SessionProperties sessionProperties, @@ -181,10 +185,10 @@ public TableRowsEntity execute( .getMaterialization(queryId, contextStacker) .orElseThrow(() -> notMaterializedException(getSourceName(analysis))); - final Struct rowKey = asKeyStruct(whereInfo.rowkey, query.getPhysicalSchema()); + final Struct key = asKeyStruct(whereInfo.keyBound, query.getPhysicalSchema()); final PullQueryContext pullQueryContext = new PullQueryContext( - rowKey, + key, mat, analysis, whereInfo, @@ -219,7 +223,7 @@ private TableRowsEntity handlePullQuery( // Get active and standby nodes for this key final Locator locator = pullQueryContext.mat.locator(); final List filteredAndOrderedNodes = locator.locate( - pullQueryContext.rowKey, + pullQueryContext.key, routingOptions, routingFilterFactory ); @@ -264,7 +268,7 @@ private TableRowsEntity routeQuery( } @VisibleForTesting - TableRowsEntity queryRowsLocally( + static TableRowsEntity queryRowsLocally( final ConfiguredStatement statement, final KsqlExecutionContext executionContext, final PullQueryContext pullQueryContext @@ -274,12 +278,12 @@ TableRowsEntity queryRowsLocally( final Range windowStart = pullQueryContext.whereInfo.windowStartBounds.get(); final List rows = pullQueryContext.mat.windowed() - .get(pullQueryContext.rowKey, windowStart); + .get(pullQueryContext.key, windowStart); result = new Result(pullQueryContext.mat.schema(), rows); } else { final List rows = pullQueryContext.mat.nonWindowed() - .get(pullQueryContext.rowKey) + .get(pullQueryContext.key) .map(ImmutableList::of) .orElse(ImmutableList.of()); @@ -371,11 +375,11 @@ TableRowsEntity forwardTo( ); } - private QueryId uniqueQueryId() { + private static QueryId uniqueQueryId() { return new QueryId("query_" + System.currentTimeMillis()); } - private ImmutableAnalysis analyze( + private static ImmutableAnalysis analyze( final ConfiguredStatement statement, final KsqlExecutionContext executionContext ) { @@ -389,7 +393,8 @@ private ImmutableAnalysis analyze( } private static final class PullQueryContext { - private final Struct rowKey; + + private final Struct key; private final Materialization mat; private final ImmutableAnalysis analysis; private final WhereInfo whereInfo; @@ -397,14 +402,14 @@ private static final class PullQueryContext { private final QueryContext.Stacker contextStacker; private PullQueryContext( - final Struct rowKey, + final Struct key, final Materialization mat, final ImmutableAnalysis analysis, final WhereInfo whereInfo, final QueryId queryId, final QueryContext.Stacker contextStacker ) { - this.rowKey = Objects.requireNonNull(rowKey, "rowkey"); + this.key = Objects.requireNonNull(key, "key"); this.mat = Objects.requireNonNull(mat, "materialization"); this.analysis = Objects.requireNonNull(analysis, "analysis"); this.whereInfo = Objects.requireNonNull(whereInfo, "whereInfo"); @@ -412,8 +417,8 @@ private PullQueryContext( this.contextStacker = Objects.requireNonNull(contextStacker, "contextStacker"); } - public Struct getRowKey() { - return rowKey; + public Struct getKey() { + return key; } public Materialization getMat() { @@ -439,14 +444,14 @@ public QueryContext.Stacker getContextStacker() { private static final class WhereInfo { - private final Object rowkey; + private final Object keyBound; private final Optional> windowStartBounds; private WhereInfo( - final Object rowkey, + final Object keyBound, final Optional> windowStartBounds ) { - this.rowkey = rowkey; + this.keyBound = keyBound; this.windowStartBounds = windowStartBounds; } } @@ -465,7 +470,7 @@ private Result( } } - private WhereInfo extractWhereInfo( + private static WhereInfo extractWhereInfo( final ImmutableAnalysis analysis, final PersistentQueryMetadata query ) { @@ -474,15 +479,16 @@ private WhereInfo extractWhereInfo( final Expression where = analysis.getWhereExpression() .orElseThrow(() -> invalidWhereClauseException("Missing WHERE clause", windowed)); - final Map> comparisons = extractComparisons(where); + final Map> comparisons = + extractComparisons(where, query); - final List rowKeyComparison = comparisons.get(ComparisonTarget.ROWKEY); - if (rowKeyComparison == null) { - throw invalidWhereClauseException("WHERE clause missing ROWKEY", windowed); + final List keyComparison = comparisons.get(ComparisonTarget.KEYCOL); + if (keyComparison == null) { + throw invalidWhereClauseException("WHERE clause missing key column", windowed); } - final Object rowKey = extractRowKeyWhereClause( - rowKeyComparison, + final Object key = extractKeyWhereClause( + keyComparison, windowed, query.getLogicalSchema() ); @@ -492,7 +498,7 @@ private WhereInfo extractWhereInfo( throw invalidWhereClauseException("Unsupported WHERE clause", false); } - return new WhereInfo(rowKey, Optional.empty()); + return new WhereInfo(key, Optional.empty()); } final Optional> windowBoundsComparison = @@ -500,30 +506,32 @@ private WhereInfo extractWhereInfo( final Range windowStart = extractWhereClauseWindowBounds(windowBoundsComparison); - return new WhereInfo(rowKey, Optional.of(windowStart)); + return new WhereInfo(key, Optional.of(windowStart)); } - private Object extractRowKeyWhereClause( + private static Object extractKeyWhereClause( final List comparisons, final boolean windowed, final LogicalSchema schema ) { if (comparisons.size() != 1) { - throw invalidWhereClauseException("Multiple bounds on ROWKEY", windowed); + throw invalidWhereClauseException("Multiple bounds on key column", windowed); } final ComparisonExpression comparison = comparisons.get(0); if (comparison.getType() != Type.EQUAL) { - throw invalidWhereClauseException("ROWKEY bound must currently be '='", windowed); + final ColumnName keyColumn = Iterables.getOnlyElement(schema.key()).name(); + throw invalidWhereClauseException("Bound on '" + keyColumn.text() + + "' must currently be '='", windowed); } final Expression other = getNonColumnRefSide(comparison); final Object right = ((Literal) other).getValue(); - return coerceRowKey(schema, right, windowed); + return coerceKey(schema, right, windowed); } - private Object coerceRowKey( + private static Object coerceKey( final LogicalSchema schema, final Object right, final boolean windowed @@ -532,14 +540,14 @@ private Object coerceRowKey( throw invalidWhereClauseException("Only single KEY column supported", windowed); } - final SqlType sqlType = schema.key().get(0).type(); + final Column keyColumn = schema.key().get(0); - return DefaultSqlValueCoercer.INSTANCE.coerce(right, sqlType) + return DefaultSqlValueCoercer.INSTANCE.coerce(right, keyColumn.type()) .orElseThrow(() -> new KsqlException("'" + right + "' can not be converted " - + "to the type of column ROWKEY: " + sqlType)); + + "to the type of the key column: " + keyColumn.toString(FormatOptions.noEscape()))); } - private Range extractWhereClauseWindowBounds( + private static Range extractWhereClauseWindowBounds( final Optional> maybeComparisons ) { if (!maybeComparisons.isPresent()) { @@ -549,7 +557,7 @@ private Range extractWhereClauseWindowBounds( final List comparisons = maybeComparisons.get(); final Map> byType = comparisons.stream() - .collect(Collectors.groupingBy(this::getSimplifiedBoundType)); + .collect(Collectors.groupingBy(PullQueryExecutor::getSimplifiedBoundType)); final SetView unsupported = Sets.difference(byType.keySet(), VALID_WINDOW_BOUNDS_TYPES); if (!unsupported.isEmpty()) { @@ -596,7 +604,7 @@ private Range extractWhereClauseWindowBounds( return extractWindowBound(lower, upper); } - private Type getSimplifiedBoundType(final ComparisonExpression comparison) { + private static Type getSimplifiedBoundType(final ComparisonExpression comparison) { final Type type = comparison.getType(); final boolean inverted = comparison.getRight() instanceof UnqualifiedColumnReferenceExp; @@ -612,7 +620,7 @@ private Type getSimplifiedBoundType(final ComparisonExpression comparison) { } } - private Range extractWindowBound( + private static Range extractWindowBound( final Optional lowerComparison, final Optional upperComparison ) { @@ -641,7 +649,7 @@ private Range extractWindowBound( return Range.range(lower, lowerType, upper, upperType); } - private BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { + private static BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { final boolean openBound = lowerComparison.getType() == Type.LESS_THAN || lowerComparison.getType() == Type.GREATER_THAN; @@ -650,13 +658,13 @@ private BoundType getRangeBoundType(final ComparisonExpression lowerComparison) : BoundType.CLOSED; } - private Expression getNonColumnRefSide(final ComparisonExpression comparison) { + private static Expression getNonColumnRefSide(final ComparisonExpression comparison) { return comparison.getRight() instanceof UnqualifiedColumnReferenceExp ? comparison.getLeft() : comparison.getRight(); } - private Instant asInstant(final Expression other) { + private static Instant asInstant(final Expression other) { if (other instanceof IntegerLiteral) { return Instant.ofEpochMilli(((IntegerLiteral) other).getValue()); } @@ -684,16 +692,17 @@ private Instant asInstant(final Expression other) { } private enum ComparisonTarget { - ROWKEY, + KEYCOL, WINDOWSTART } - private Map> extractComparisons( - final Expression exp + private static Map> extractComparisons( + final Expression exp, + final PersistentQueryMetadata query ) { if (exp instanceof ComparisonExpression) { final ComparisonExpression comparison = (ComparisonExpression) exp; - final ComparisonTarget target = extractWhereClauseTarget(comparison); + final ComparisonTarget target = extractWhereClauseTarget(comparison, query); return ImmutableMap.of(target, ImmutableList.of(comparison)); } @@ -704,10 +713,10 @@ private Map> extractComparisons( } final Map> left = - extractComparisons(binary.getLeft()); + extractComparisons(binary.getLeft(), query); final Map> right = - extractComparisons(binary.getRight()); + extractComparisons(binary.getRight(), query); return Stream .concat(left.entrySet().stream(), right.entrySet().stream()) @@ -719,7 +728,10 @@ private Map> extractComparisons( throw invalidWhereClauseException("Unsupported expression: " + exp, false); } - private ComparisonTarget extractWhereClauseTarget(final ComparisonExpression comparison) { + private static ComparisonTarget extractWhereClauseTarget( + final ComparisonExpression comparison, + final PersistentQueryMetadata query + ) { final UnqualifiedColumnReferenceExp column; if (comparison.getRight() instanceof UnqualifiedColumnReferenceExp) { column = (UnqualifiedColumnReferenceExp) comparison.getRight(); @@ -729,21 +741,28 @@ private ComparisonTarget extractWhereClauseTarget(final ComparisonExpression com throw invalidWhereClauseException("Invalid WHERE clause: " + comparison, false); } - final String fieldName = column.getColumnName().toString(FormatOptions.noEscape()); + final ColumnName columnName = column.getColumnName(); + if (columnName.equals(SchemaUtil.WINDOWSTART_NAME)) { + return ComparisonTarget.WINDOWSTART; + } - try { - return ComparisonTarget.valueOf(fieldName.toUpperCase()); - } catch (final Exception e) { - throw invalidWhereClauseException("WHERE clause on unsupported field: " + fieldName, false); + final ColumnName keyColumn = Iterables.getOnlyElement(query.getLogicalSchema().key()).name(); + if (columnName.equals(keyColumn)) { + return ComparisonTarget.KEYCOL; } + + throw invalidWhereClauseException( + "WHERE clause on unsupported column: " + columnName.text(), + false + ); } - private boolean isSelectStar(final Select select) { + private static boolean isSelectStar(final Select select) { final List selects = select.getSelectItems(); return selects.size() == 1 && selects.get(0) instanceof AllColumns; } - private List> handleSelects( + private static List> handleSelects( final Result input, final ConfiguredStatement statement, final KsqlExecutionContext executionContext, @@ -831,7 +850,7 @@ private List> handleSelects( return output.build(); } - private void validateProjection( + private static void validateProjection( final GenericRow fullRow, final LogicalSchema schema ) { @@ -876,7 +895,7 @@ private static LogicalSchema selectOutputSchema( return schemaBuilder.build(); } - private PersistentQueryMetadata findMaterializingQuery( + private static PersistentQueryMetadata findMaterializingQuery( final KsqlExecutionContext executionContext, final ImmutableAnalysis analysis ) { @@ -900,12 +919,12 @@ private PersistentQueryMetadata findMaterializingQuery( .orElseThrow(() -> new KsqlException("Materializing query has been stopped")); } - private SourceName getSourceName(final ImmutableAnalysis analysis) { + private static SourceName getSourceName(final ImmutableAnalysis analysis) { final DataSource source = analysis.getFromDataSources().get(0).getDataSource(); return source.getName(); } - private KsqlException notMaterializedException(final SourceName sourceTable) { + private static KsqlException notMaterializedException(final SourceName sourceTable) { return new KsqlException("'" + sourceTable.toString(FormatOptions.noEscape()) + "' is not materialized. " + PullQueryValidator.NEW_QUERY_SYNTAX_SHORT_HELP @@ -918,7 +937,7 @@ private KsqlException notMaterializedException(final SourceName sourceTable) { ); } - private KsqlException invalidWhereClauseException( + private static KsqlException invalidWhereClauseException( final String msg, final boolean windowed ) { @@ -942,19 +961,23 @@ private KsqlException invalidWhereClauseException( + System.lineSeparator() + "Pull queries require a WHERE clause that:" + System.lineSeparator() - + " - limits the query to a single ROWKEY, e.g. `SELECT * FROM X WHERE ROWKEY=Y;`." + + " - limits the query to a single key, e.g. `SELECT * FROM X WHERE >=Y;`." + additional ); } - private Struct asKeyStruct(final Object rowKey, final PhysicalSchema physicalSchema) { + private static Struct asKeyStruct(final Object keyValue, final PhysicalSchema physicalSchema) { + final Field keyField = Iterables + .getOnlyElement(physicalSchema.keySchema().ksqlSchema().fields()); + final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); - key.put(SchemaUtil.ROWKEY_NAME.text(), rowKey); + key.put(keyField, keyValue); return key; } private static final class ColumnReferenceRewriter extends VisitParentExpressionVisitor, Context> { + private ColumnReferenceRewriter() { super(Optional.empty()); } @@ -1007,7 +1030,5 @@ public long getOffsetLagAllowed() { public boolean skipForwardRequest() { return getForwardedFlag(KsqlRequestConfig.KSQL_REQUEST_QUERY_PULL_SKIP_FORWARDING); } - - } } \ No newline at end of file diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java index 1c7f5a6b1904..09934028351d 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java @@ -55,7 +55,7 @@ public final class GenericRowSerDe implements ValueSerdeFactory { *

Count covers the following additional columns: *

    *
  1. {@link SchemaUtil#ROWTIME_NAME}
  2. - *
  3. {@link SchemaUtil#ROWKEY_NAME}
  4. + *
  5. A single key column. (Which is the most common case)
  6. *
  7. {@link SchemaUtil#WINDOWSTART_NAME}
  8. *
  9. {@link SchemaUtil#WINDOWEND_NAME}
  10. *
diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java index 1df35ef2cf1e..6e829bf1a2e1 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/GroupByParamsFactory.java @@ -75,18 +75,19 @@ private static LogicalSchema multiExpressionSchema( private static LogicalSchema singleExpressionSchema( final LogicalSchema sourceSchema, - final SqlType rowKeyType + final SqlType keyType ) { - return buildSchemaWithKeyType(sourceSchema, rowKeyType); + return buildSchemaWithKeyType(sourceSchema, keyType); } private static LogicalSchema buildSchemaWithKeyType( final LogicalSchema sourceSchema, - final SqlType rowKeyType + final SqlType keyType ) { + return LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, rowKeyType) + .keyColumn(SchemaUtil.ROWKEY_NAME, keyType) .valueColumns(sourceSchema.value()) .build(); } @@ -136,16 +137,17 @@ private static final class SingleExpressionGrouper { final ExpressionMetadata expression, final ProcessingLogger logger) { this.expression = requireNonNull(expression, "expression"); - this.keyBuilder = StructKeyUtil.keyBuilder(expression.getExpressionType()); + this.keyBuilder = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, expression.getExpressionType()); this.logger = Objects.requireNonNull(logger, "logger"); } public Struct apply(final GenericRow row) { - final Object rowKey = processColumn(0, expression, row, logger); - if (rowKey == null) { + final Object key = processColumn(0, expression, row, logger); + if (key == null) { return null; } - return keyBuilder.build(rowKey); + return keyBuilder.build(key); } } @@ -160,7 +162,7 @@ private static final class MultiExpressionGrouper { final ProcessingLogger logger ) { this.expressions = ImmutableList.copyOf(requireNonNull(expressions, "expressions")); - this.keyBuilder = StructKeyUtil.keyBuilder(SqlTypes.STRING); + this.keyBuilder = StructKeyUtil.keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING); this.logger = Objects.requireNonNull(logger, "logger"); if (expressions.isEmpty()) { @@ -169,21 +171,21 @@ private static final class MultiExpressionGrouper { } public Struct apply(final GenericRow row) { - final StringBuilder rowKey = new StringBuilder(); + final StringBuilder key = new StringBuilder(); for (int i = 0; i < expressions.size(); i++) { final Object result = processColumn(i, expressions.get(i), row, logger); if (result == null) { return null; } - if (rowKey.length() > 0) { - rowKey.append(GROUP_BY_VALUE_SEPARATOR); + if (key.length() > 0) { + key.append(GROUP_BY_VALUE_SEPARATOR); } - rowKey.append(result); + key.append(result); } - return keyBuilder.build(rowKey.toString()); + return keyBuilder.build(key.toString()); } } } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java index 86e8b5569ae2..d830842be011 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SourceBuilder.java @@ -101,7 +101,7 @@ public static KStreamHolder buildStream( source, queryBuilder, consumed, - nonWindowedRowKeyGenerator(source.getSourceSchema()) + nonWindowedKeyGenerator(source.getSourceSchema()) ); return new KStreamHolder<>( @@ -141,7 +141,7 @@ static KStreamHolder> buildWindowedStream( source, queryBuilder, consumed, - windowedRowKeyGenerator(source.getSourceSchema()) + windowedKeyGenerator(source.getSourceSchema()) ); return new KStreamHolder<>( @@ -187,7 +187,7 @@ public static KTableHolder buildTable( source, queryBuilder, consumed, - nonWindowedRowKeyGenerator(source.getSourceSchema()), + nonWindowedKeyGenerator(source.getSourceSchema()), materialized ); @@ -236,7 +236,7 @@ static KTableHolder> buildWindowedTable( source, queryBuilder, consumed, - windowedRowKeyGenerator(source.getSourceSchema()), + windowedKeyGenerator(source.getSourceSchema()), materialized ); @@ -278,27 +278,27 @@ private static KStream buildKStream( final SourceStep streamSource, final KsqlQueryBuilder queryBuilder, final Consumed consumed, - final Function> rowKeyGenerator + final Function> keyGenerator ) { final KStream stream = queryBuilder.getStreamsBuilder() .stream(streamSource.getTopicName(), consumed); return stream - .transformValues(new AddKeyAndTimestampColumns<>(rowKeyGenerator)); + .transformValues(new AddKeyAndTimestampColumns<>(keyGenerator)); } private static KTable buildKTable( final SourceStep streamSource, final KsqlQueryBuilder queryBuilder, final Consumed consumed, - final Function> rowKeyGenerator, + final Function> keyGenerator, final Materialized> materialized ) { final KTable table = queryBuilder.getStreamsBuilder() .table(streamSource.getTopicName(), consumed, materialized); return table - .transformValues(new AddKeyAndTimestampColumns<>(rowKeyGenerator)); + .transformValues(new AddKeyAndTimestampColumns<>(keyGenerator)); } private static TimestampExtractor timestampExtractor( @@ -369,7 +369,7 @@ private static String tableChangeLogOpName(final ExecutionStepPropertiesV1 props return StreamsUtil.buildOpName(stacker.push("Reduce").getQueryContext()); } - private static Function, Collection> windowedRowKeyGenerator( + private static Function, Collection> windowedKeyGenerator( final LogicalSchema schema ) { final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); @@ -385,7 +385,7 @@ private static Function, Collection> windowedRowKeyGenerator }; } - private static Function> nonWindowedRowKeyGenerator( + private static Function> nonWindowedKeyGenerator( final LogicalSchema schema ) { final org.apache.kafka.connect.data.Field keyField = getKeySchemaSingleField(schema); @@ -401,10 +401,10 @@ private static Function> nonWindowedRowKeyGenerator( private static class AddKeyAndTimestampColumns implements ValueTransformerWithKeySupplier { - private final Function> rowKeyGenerator; + private final Function> keyGenerator; - AddKeyAndTimestampColumns(final Function> rowKeyGenerator) { - this.rowKeyGenerator = requireNonNull(rowKeyGenerator, "rowKeyGenerator"); + AddKeyAndTimestampColumns(final Function> keyGenerator) { + this.keyGenerator = requireNonNull(keyGenerator, "keyGenerator"); } @Override @@ -424,7 +424,7 @@ public GenericRow transform(final K key, final GenericRow row) { } final long timestamp = processorContext.timestamp(); - final Collection keyColumns = rowKeyGenerator.apply(key); + final Collection keyColumns = keyGenerator.apply(key); row.ensureAdditionalCapacity(1 + keyColumns.size()); row.append(timestamp); From 6c6b03af5e7792f9890256e71ed15b52fdcdf224 Mon Sep 17 00:00:00 2001 From: Andy Coates Date: Wed, 4 Mar 2020 13:41:11 +0000 Subject: [PATCH 2/3] chore: commit test files --- .../ksql/analyzer/SourceSchemasTest.java | 42 +- .../ksql/codegen/CodeGenRunnerTest.java | 3 +- .../ddl/commands/CreateSourceFactoryTest.java | 19 +- .../ksql/ddl/commands/DdlCommandExecTest.java | 3 +- .../ksql/engine/InsertValuesExecutorTest.java | 124 +- .../SelectValueMapperIntegrationTest.java | 4 +- .../physical/PhysicalPlanBuilderTest.java | 4 +- .../ksql/planner/plan/DataSourceNodeTest.java | 6 +- .../ksql/planner/plan/FilterNodeTest.java | 9 +- .../ksql/planner/plan/JoinNodeTest.java | 25 +- .../KsqlStructuredDataOutputNodeTest.java | 58 +- .../ksql/structured/SchemaKTableTest.java | 4 +- .../ddl/commands/CreateSourceCommandTest.java | 14 +- .../transform/select/SelectionTest.java | 7 +- .../execution/util/StructKeyUtilTest.java | 8 +- ksql-functional-tests/README.md | 6 +- .../query-validation-tests/elements.json | 73 +- .../query-validation-tests/group-by.json | 112 +- .../query-validation-tests/joins.json | 1662 ++++++++++++++++- .../query-validation-tests/key-field.json | 2 +- .../query-validation-tests/key-schemas.json | 55 +- .../query-validation-tests/partition-by.json | 85 +- .../query-validation-tests/select.json | 91 + .../system-columns.json | 12 +- .../table-functions.json | 21 + .../query-validation-tests/where.json | 57 + .../insert-values.json | 444 ++++- ...eries-against-materialized-aggregates.json | 35 +- .../model/StructuredDataSourceTest.java | 31 +- .../confluent/ksql/parser/KsqlParserTest.java | 3 +- .../ksql/parser/SchemaParserTest.java | 17 +- .../ksql/parser/SqlFormatterTest.java | 15 +- .../ksql/parser/tree/TableElementTest.java | 4 +- .../ksql/parser/tree/TableElementsTest.java | 20 + .../api/integration/ApiIntegrationTest.java | 2 +- .../entity/TableRowsEntityFactoryTest.java | 23 +- .../streams/GroupByParamsFactoryTest.java | 7 +- .../streams/JoinParamsFactoryTest.java | 7 +- .../execution/streams/SourceBuilderTest.java | 9 +- .../streams/StepSchemaResolverTest.java | 18 +- .../streams/StreamGroupByBuilderTest.java | 6 +- .../streams/StreamSelectKeyBuilderTest.java | 9 +- .../streams/TableGroupByBuilderTest.java | 5 +- .../KsqlMaterializationTest.java | 6 +- .../ks/KsMaterializedSessionTableTest.java | 5 +- .../ks/KsMaterializedTableTest.java | 5 +- .../ks/KsMaterializedWindowTableTest.java | 6 +- 47 files changed, 2904 insertions(+), 279 deletions(-) create mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/select.json create mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/where.json diff --git a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java index e55d889d7fc3..0d0587e3c4ea 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/SourceSchemasTest.java @@ -35,20 +35,28 @@ public class SourceSchemasTest { private static final SourceName ALIAS_1 = SourceName.of("S1"); private static final SourceName ALIAS_2 = SourceName.of("S2"); - private static final ColumnName COMMON_FIELD_NAME = ColumnName.of("F0"); + + private static final ColumnName K0 = ColumnName.of("K0"); + private static final ColumnName K1 = ColumnName.of("K1"); + + private static final ColumnName COMMON_VALUE_FIELD_NAME = ColumnName.of("V0"); + private static final ColumnName V1 = ColumnName.of("V1"); + private static final ColumnName V2 = ColumnName.of("V2"); private static final LogicalSchema SCHEMA_1 = LogicalSchema.builder() .withRowTime() .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) - .valueColumn(COMMON_FIELD_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("F1"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.INTEGER) + .valueColumn(COMMON_VALUE_FIELD_NAME, SqlTypes.STRING) + .valueColumn(V1, SqlTypes.STRING) .build(); private static final LogicalSchema SCHEMA_2 = LogicalSchema.builder() .withRowTime() .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) - .valueColumn(COMMON_FIELD_NAME, SqlTypes.STRING) - .valueColumn(ColumnName.of("F2"), SqlTypes.STRING) + .keyColumn(K1, SqlTypes.STRING) + .valueColumn(COMMON_VALUE_FIELD_NAME, SqlTypes.STRING) + .valueColumn(V2, SqlTypes.STRING) .build(); private SourceSchemas sourceSchemas; @@ -88,28 +96,28 @@ public void shouldFindNoField() { @Test public void shouldFindNoQualifiedField() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), ColumnName.of("F2")), is(empty())); + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), V2), is(empty())); } @Test public void shouldFindUnqualifiedUniqueField() { - assertThat(sourceSchemas.sourcesWithField(Optional.empty(), ColumnName.of("F1")), contains(ALIAS_1)); + assertThat(sourceSchemas.sourcesWithField(Optional.empty(), V1), contains(ALIAS_1)); } @Test public void shouldFindQualifiedUniqueField() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_2), ColumnName.of("F2")), contains(ALIAS_2)); + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_2), V2), contains(ALIAS_2)); } @Test public void shouldFindUnqualifiedCommonField() { - assertThat(sourceSchemas.sourcesWithField(Optional.empty(), COMMON_FIELD_NAME), + assertThat(sourceSchemas.sourcesWithField(Optional.empty(), COMMON_VALUE_FIELD_NAME), containsInAnyOrder(ALIAS_1, ALIAS_2)); } @Test public void shouldFindQualifiedFieldOnlyInThatSource() { - assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), COMMON_FIELD_NAME), + assertThat(sourceSchemas.sourcesWithField(Optional.of(ALIAS_1), COMMON_VALUE_FIELD_NAME), contains(ALIAS_1)); } @@ -125,22 +133,28 @@ public void shouldMatchNonValueFieldNameIfAliaasedMetaField() { @Test public void shouldMatchNonValueFieldNameIfKeyField() { - assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), SchemaUtil.ROWKEY_NAME), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), K0), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.empty(), K1), is(true)); + } + + @Test + public void shouldNotMatchNonKeyFieldOnWrongSource() { + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), K0), is(false)); } @Test public void shouldMatchNonValueFieldNameIfAliasedKeyField() { - assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), SchemaUtil.ROWKEY_NAME), is(true)); + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), K1), is(true)); } @Test(expected = IllegalArgumentException.class) public void shouldThrowOnUnknownSourceWhenMatchingNonValueFields() { - sourceSchemas.matchesNonValueField(Optional.of(SourceName.of("unknown")), SchemaUtil.ROWKEY_NAME); + sourceSchemas.matchesNonValueField(Optional.of(SourceName.of("unknown")), K0); } @Test public void shouldNotMatchOtherFields() { - assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), ColumnName.of("F2")), is(false)); + assertThat(sourceSchemas.matchesNonValueField(Optional.of(ALIAS_2), V2), is(false)); } @Test diff --git a/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java b/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java index f25e6fb78dcf..e7b7759e7186 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/codegen/CodeGenRunnerTest.java @@ -66,7 +66,6 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collections; @@ -88,7 +87,7 @@ public class CodeGenRunnerTest { private static final String COL_INVALID_JAVA = "col!Invalid:("; private static final LogicalSchema META_STORE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("COL0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("COL1"), SqlTypes.STRING) .valueColumn(ColumnName.of("COL2"), SqlTypes.STRING) diff --git a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java index af200165c82d..37f5e2b7cb28 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java @@ -863,8 +863,12 @@ public void shouldAllowNonStringKeyColumn() { } @Test - public void shouldThrowOnKeyColumnThatIsNotCalledRowKey() { + public void shouldNotThrowOnKeyColumnThatIsNotCalledRowKey() { // Given: + ksqlConfig = new KsqlConfig(ImmutableMap.of( + KsqlConfig.KSQL_ANY_KEY_NAME_ENABLED, true + )); + final CreateStream statement = new CreateStream( SOME_NAME, TableElements.of(tableElement(KEY, "someKey", new Type(SqlTypes.STRING))), @@ -872,13 +876,14 @@ public void shouldThrowOnKeyColumnThatIsNotCalledRowKey() { withProperties ); - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("'someKey' is an invalid KEY column name. " - + "KSQL currently only supports KEY columns named ROWKEY."); - // When: - createSourceFactory.createStreamCommand(statement, ksqlConfig); + final CreateStreamCommand result = createSourceFactory + .createStreamCommand(statement, ksqlConfig); + + // Then: + assertThat(result.getSchema().key(), contains( + keyColumn(ColumnName.of("someKey"), SqlTypes.STRING) + )); } private void givenProperty(final String name, final Literal value) { diff --git a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java index 699edb9a4efd..a3e41061ffd7 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/DdlCommandExecTest.java @@ -28,7 +28,6 @@ import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.serde.WindowInfo; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import java.util.Set; import org.hamcrest.MatcherAssert; @@ -46,7 +45,7 @@ public class DdlCommandExecTest { private static final SourceName TABLE_NAME = SourceName.of("t1"); private static final String TOPIC_NAME = "topic"; private static final LogicalSchema SCHEMA = new LogicalSchema.Builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("F1"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("F2"), SqlTypes.STRING) .build(); diff --git a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java index 8774c503c4eb..31ee1c6dc4a6 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.GenericRow; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; @@ -98,25 +99,29 @@ @RunWith(MockitoJUnitRunner.class) public class InsertValuesExecutorTest { + private static final ColumnName K0 = ColumnName.of("k0"); private static final ColumnName COL0 = ColumnName.of("COL0"); - private static final LogicalSchema SINGLE_FIELD_SCHEMA = LogicalSchema.builder() + private static final ColumnName COL1 = ColumnName.of("COL1"); + private static final ColumnName INT_COL = ColumnName.of("INT"); + + private static final LogicalSchema SINGLE_VALUE_COLUMN_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .valueColumn(COL0, SqlTypes.STRING) .build(); private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .valueColumn(COL0, SqlTypes.STRING) - .valueColumn(ColumnName.of("COL1"), SqlTypes.BIGINT) + .valueColumn(COL1, SqlTypes.BIGINT) .build(); private static final LogicalSchema BIG_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .valueColumn(COL0, SqlTypes.STRING) // named COL0 for auto-ROWKEY - .valueColumn(ColumnName.of("INT"), SqlTypes.INTEGER) + .valueColumn(INT_COL, SqlTypes.INTEGER) .valueColumn(ColumnName.of("BIGINT"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("DOUBLE"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("BOOLEAN"), SqlTypes.BOOLEAN) @@ -210,10 +215,10 @@ public void shouldHandleFullRow() { @Test public void shouldInsertWrappedSingleField() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.of(COL0)); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.of(COL0)); final ConfiguredStatement statement = givenInsertValues( - valueFieldNames(SINGLE_FIELD_SCHEMA), + valueFieldNames(SINGLE_VALUE_COLUMN_SCHEMA), ImmutableList.of(new StringLiteral("new")) ); @@ -230,13 +235,13 @@ public void shouldInsertWrappedSingleField() { public void shouldInsertUnwrappedSingleField() { // Given: givenSourceStreamWithSchema( - SINGLE_FIELD_SCHEMA, + SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.of(SerdeOption.UNWRAP_SINGLE_VALUES), Optional.of(COL0)) ; final ConfiguredStatement statement = givenInsertValues( - valueFieldNames(SINGLE_FIELD_SCHEMA), + valueFieldNames(SINGLE_VALUE_COLUMN_SCHEMA), ImmutableList.of(new StringLiteral("new")) ); @@ -252,8 +257,8 @@ public void shouldInsertUnwrappedSingleField() { @Test public void shouldFillInRowtime() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("str"), new StringLiteral("str"), @@ -271,10 +276,10 @@ public void shouldFillInRowtime() { } @Test - public void shouldHandleRowTimeWithoutRowKey() { + public void shouldHandleRowTimeWithoutKey() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWTIME", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(SchemaUtil.ROWTIME_NAME, COL0, COL1), ImmutableList.of( new LongLiteral(1234L), new StringLiteral("str"), @@ -292,10 +297,10 @@ public void shouldHandleRowTimeWithoutRowKey() { } @Test - public void shouldFillInRowKeyFromSpecifiedKey() { + public void shouldFillInKeyColumnFromSpecifiedKeyField() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L) @@ -335,8 +340,8 @@ public void shouldFillInFullRowWithNoSchema() { @Test public void shouldFillInMissingColumnsWithNulls() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0), ImmutableList.of( new StringLiteral("str"), new StringLiteral("str")) @@ -354,8 +359,8 @@ public void shouldFillInMissingColumnsWithNulls() { @Test public void shouldFillInKeyFromRowKey() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L) @@ -374,8 +379,8 @@ public void shouldFillInKeyFromRowKey() { @Test public void shouldHandleOutOfOrderSchema() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1, COL0), ImmutableList.of( new LongLiteral(2L), new StringLiteral("str") @@ -394,8 +399,8 @@ public void shouldHandleOutOfOrderSchema() { @Test public void shouldHandleAllSortsOfLiterals() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1, COL0), ImmutableList.of( new LongLiteral(2L), new StringLiteral("str")) @@ -445,10 +450,10 @@ public void shouldHandleNullKeyForSourceWithKeyField() { @Test public void shouldHandleNegativeValueExpression() { // Given: - givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(ColumnName.of("COL0"))); + givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), ArithmeticUnaryExpression.negative(Optional.empty(), new LongLiteral(1)) @@ -467,10 +472,10 @@ public void shouldHandleNegativeValueExpression() { @Test public void shouldHandleUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0), ImmutableList.of( new FunctionCall( FunctionName.of("SUBSTRING"), @@ -488,10 +493,10 @@ public void shouldHandleUdfs() { @Test public void shouldHandleNestedUdfs() { // Given: - givenSourceStreamWithSchema(SINGLE_FIELD_SCHEMA, SerdeOption.none(), Optional.empty()); + givenSourceStreamWithSchema(SINGLE_VALUE_COLUMN_SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0), ImmutableList.of( new FunctionCall( FunctionName.of("SUBSTRING"), @@ -517,8 +522,8 @@ public void shouldAllowUpcast() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new IntegerLiteral(1) @@ -689,8 +694,8 @@ public void shouldThrowOnTopicAuthorizationException() { @Test public void shouldThrowIfRowKeyAndKeyDoNotMatch() { // Given: - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0), ImmutableList.of( new StringLiteral("foo"), new StringLiteral("bar")) @@ -698,7 +703,7 @@ public void shouldThrowIfRowKeyAndKeyDoNotMatch() { // Expect: expectedException.expect(KsqlException.class); - expectedException.expectCause(hasMessage(containsString("Expected ROWKEY and COL0 to match"))); + expectedException.expectCause(hasMessage(containsString("Expected k0 and COL0 to match"))); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -726,8 +731,8 @@ public void shouldFailOnDowncast() { // Given: givenSourceStreamWithSchema(BIG_SCHEMA, SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("INT"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(INT_COL), ImmutableList.of( new DoubleLiteral(1.1) ) @@ -746,8 +751,8 @@ public void shouldHandleStreamsWithNoKeyField() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("key"), new StringLiteral("str"), @@ -768,8 +773,8 @@ public void shouldHandleTablesWithNoKeyField() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("ROWKEY", "COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(K0, COL0, COL1), ImmutableList.of( new StringLiteral("key"), new StringLiteral("str"), @@ -790,8 +795,8 @@ public void shouldHandleStreamsWithNoKeyFieldAndNoRowKeyProvided() { // Given: givenSourceStreamWithSchema(SCHEMA, SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L)) @@ -811,8 +816,8 @@ public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.empty()); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL0", "COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL0, COL1), ImmutableList.of( new StringLiteral("str"), new LongLiteral(2L)) @@ -821,7 +826,7 @@ public void shouldThrowOnTablesWithNoKeyFieldAndNoRowKeyProvided() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage( - "Failed to insert values into 'TOPIC'. Value for ROWKEY is required for tables"); + "Failed to insert values into 'TOPIC'. Value for primary key column(s) k0 is required for tables"); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -832,8 +837,8 @@ public void shouldThrowOnTablesWithKeyFieldAndNullKeyFieldValueProvided() { // Given: givenSourceTableWithSchema(SerdeOption.none(), Optional.of(COL0)); - final ConfiguredStatement statement = givenInsertValuesStrings( - ImmutableList.of("COL1"), + final ConfiguredStatement statement = givenInsertValues( + ImmutableList.of(COL1), ImmutableList.of( new LongLiteral(2L)) ); @@ -841,7 +846,7 @@ public void shouldThrowOnTablesWithKeyFieldAndNullKeyFieldValueProvided() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage( - "Failed to insert values into 'TOPIC'. Value for ROWKEY is required for tables"); + "Failed to insert values into 'TOPIC'. Value for primary key column(s) k0 is required for tables"); // When: executor.execute(statement, mock(SessionProperties.class), engine, serviceContext); @@ -881,13 +886,6 @@ public void shouldBuildCorrectSerde() { ); } - private static ConfiguredStatement givenInsertValuesStrings( - final List columns, - final List values - ) { - return givenInsertValues(columns.stream().map(ColumnName::of).collect(Collectors.toList()), values); - } - private static ConfiguredStatement givenInsertValues( final List columns, final List values @@ -930,7 +928,7 @@ private void givenDataSourceWithSchema( ); final KeyField valueKeyField = keyField - .map(kf -> KeyField.of(kf)) + .map(KeyField::of) .orElse(KeyField.none()); final DataSource dataSource; @@ -966,7 +964,7 @@ private void givenDataSourceWithSchema( private static Struct keyStruct(final String rowKey) { final Struct key = new Struct(SCHEMA.keyConnectSchema()); - key.put("ROWKEY", rowKey); + key.put(Iterables.getOnlyElement(SCHEMA.key()).name().text(), rowKey); return key; } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java b/ksql-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java index 59a34d1a3564..8a223bd784d1 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/integration/SelectValueMapperIntegrationTest.java @@ -28,6 +28,7 @@ import io.confluent.ksql.function.TestFunctionRegistry; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.MetaStore; +import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.planner.plan.PlanNode; import io.confluent.ksql.planner.plan.ProjectNode; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -46,7 +47,8 @@ public class SelectValueMapperIntegrationTest { - private static final Struct NON_WINDOWED_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING) + private static final Struct NON_WINDOWED_KEY = StructKeyUtil + .keyBuilder(ColumnName.of("K"), SqlTypes.STRING) .build("someKey"); private final MetaStore metaStore = MetaStoreFixture diff --git a/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java b/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java index 2247a6df51ee..f821ab36c34d 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java @@ -354,7 +354,7 @@ public void shouldThrowIfLeftTableNotJoiningOnTableKey() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage("Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE key `ID` instead of COL0"); + + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0"); // When: execute("CREATE TABLE t1 AS " @@ -371,7 +371,7 @@ public void shouldThrowIfRightTableNotJoiningOnTableKey() { // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage("Cannot repartition a TABLE source. If this is a join, make " - + "sure that the criteria uses the TABLE key `ID` instead of COL0"); + + "sure that the criteria uses the TABLE's key column ROWKEY instead of COL0"); // When: execute("CREATE TABLE t1 AS " diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java index d07c6dbe1f8a..719f744d376a 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/DataSourceNodeTest.java @@ -59,7 +59,6 @@ import io.confluent.ksql.structured.SchemaKStream; import io.confluent.ksql.structured.SchemaKTable; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.SchemaUtil; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -80,8 +79,7 @@ @RunWith(MockitoJUnitRunner.class) public class DataSourceNodeTest { - private static final ColumnName TIMESTAMP_FIELD - = ColumnName.of("timestamp"); + private static final ColumnName TIMESTAMP_FIELD = ColumnName.of("timestamp"); private static final PlanNodeId PLAN_NODE_ID = new PlanNodeId("0"); private static final SourceName SOURCE_NAME = SourceName.of("datasource"); @@ -94,7 +92,7 @@ public class DataSourceNodeTest { private static final ColumnName FIELD3 = ColumnName.of("field3"); private static final LogicalSchema REAL_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.INTEGER) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(FIELD1, SqlTypes.INTEGER) .valueColumn(FIELD2, SqlTypes.STRING) .valueColumn(FIELD3, SqlTypes.STRING) diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/FilterNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/FilterNodeTest.java index 305c6f1769e7..f51ee2c27907 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/FilterNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/FilterNodeTest.java @@ -25,6 +25,7 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext.Stacker; import io.confluent.ksql.execution.expression.tree.Expression; +import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.structured.SchemaKStream; import org.junit.Before; import org.junit.Rule; @@ -34,7 +35,8 @@ import org.mockito.junit.MockitoRule; public class FilterNodeTest { - private final PlanNodeId nodeId = new PlanNodeId("nodeid"); + + private static final PlanNodeId NODE_ID = new PlanNodeId("nodeid"); @Mock private Expression predicate; @@ -55,15 +57,16 @@ public class FilterNodeTest { @Before @SuppressWarnings("unchecked") public void setup() { + when(sourceNode.getSchema()).thenReturn(LogicalSchema.builder().build()); when(sourceNode.buildStream(any())) .thenReturn(schemaKStream); when(sourceNode.getNodeOutputType()).thenReturn(DataSourceType.KSTREAM); when(schemaKStream.filter(any(), any())) .thenReturn(schemaKStream); - when(ksqlStreamBuilder.buildNodeContext(nodeId.toString())).thenReturn(stacker); + when(ksqlStreamBuilder.buildNodeContext(NODE_ID.toString())).thenReturn(stacker); - node = new FilterNode(nodeId, sourceNode, predicate); + node = new FilterNode(NODE_ID, sourceNode, predicate); } @Test diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java index eed91333a0d1..566dfc9d4dfd 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java @@ -60,7 +60,6 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -89,14 +88,14 @@ public class JoinNodeTest { private static final LogicalSchema LEFT_SOURCE_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("C0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("L1"), SqlTypes.STRING) .build(); private static final LogicalSchema RIGHT_SOURCE_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("rightKey"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("C0"), SqlTypes.STRING) .valueColumn(ColumnName.of("R1"), SqlTypes.BIGINT) .build(); @@ -647,15 +646,15 @@ public void shouldHaveFullyQualifiedJoinSchema() { // When: assertThat(joinNode.getSchema(), is(LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "C0"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "L1"), SqlTypes.STRING) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "ROWTIME"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_" + "ROWKEY"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "C0"), SqlTypes.STRING) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "R1"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "ROWTIME"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_" + "ROWKEY"), SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_C0"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_L1"), SqlTypes.STRING) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_ROWTIME"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(LEFT_ALIAS.text() + "_leftKey"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_C0"), SqlTypes.STRING) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_R1"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_ROWTIME"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of(RIGHT_ALIAS.text() + "_rightKey"), SqlTypes.BIGINT) .build() )); } @@ -698,7 +697,7 @@ public void shouldReturnCorrectSchema() { // Then: assertThat(joinNode.getSchema(), is(LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("leftKey"), SqlTypes.BIGINT) .valueColumns(LEFT_NODE_SCHEMA.value()) .valueColumns(RIGHT_NODE_SCHEMA.value()) .build())); diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java index b01ae8115069..ed08b5c14bf5 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.sameInstance; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -29,6 +30,7 @@ import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; +import io.confluent.ksql.execution.plan.SelectExpression; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; @@ -43,7 +45,7 @@ import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.serde.avro.AvroFormat; import io.confluent.ksql.structured.SchemaKStream; -import io.confluent.ksql.util.SchemaUtil; +import io.confluent.ksql.util.KsqlException; import java.util.Optional; import java.util.OptionalInt; import org.junit.Before; @@ -56,18 +58,16 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; -@SuppressWarnings("OptionalGetWithoutIsPresent") @RunWith(MockitoJUnitRunner.class) public class KsqlStructuredDataOutputNodeTest { private static final String QUERY_ID_VALUE = "output-test"; - private static final QueryId QUERY_ID = new QueryId(QUERY_ID_VALUE); private static final String SINK_KAFKA_TOPIC_NAME = "output_kafka"; private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("field1"), SqlTypes.STRING) .valueColumn(ColumnName.of("field2"), SqlTypes.STRING) .valueColumn(ColumnName.of("field3"), SqlTypes.STRING) @@ -93,26 +93,18 @@ public class KsqlStructuredDataOutputNodeTest { @Mock private SchemaKStream sourceStream; @Mock - private SchemaKStream resultStream; - @Mock private SchemaKStream sinkStream; @Mock - private SchemaKStream resultWithKeySelected; - @Mock - private SchemaKStream sinkStreamWithKeySelected; - @Mock private KsqlTopic ksqlTopic; @Captor private ArgumentCaptor stackerCaptor; private KsqlStructuredDataOutputNode outputNode; - private LogicalSchema schema; private boolean createInto; - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "rawtypes"}) @Before public void before() { - schema = SCHEMA; createInto = true; when(queryIdGenerator.getNext()).thenReturn(QUERY_ID_VALUE); @@ -132,6 +124,24 @@ public void before() { buildNode(); } + @Test + public void shouldThrowIfSelectExpressionsHaveSameNameAsAnyKeyColumn() { + // Given: + givenSourceSelectExpressions( + selectExpression("field1"), + selectExpression("k0"), + selectExpression("field2") + ); + + // Expect: + expectedException.expect(KsqlException.class); + expectedException.expectMessage("Value column name(s) `k0` " + + "clashes with key column name(s). Please remove or alias the column(s)."); + + // When: + buildNode(); + } + @Test public void shouldBuildSourceNode() { // When: @@ -155,7 +165,7 @@ public void shouldComputeQueryIdCorrectlyForStream() { public void shouldComputeQueryIdCorrectlyForTable() { // Given: when(sourceNode.getNodeOutputType()).thenReturn(DataSourceType.KTABLE); - givenNodeWithSchema(SCHEMA); + buildNode(); // When: final QueryId queryId = outputNode.getQueryId(queryIdGenerator); @@ -198,7 +208,7 @@ public void shouldBuildOutputNodeForInsertIntoAvroFromNonAvro() { @Test public void shouldCallInto() { // When: - final SchemaKStream result = outputNode.buildStream(ksqlStreamBuilder); + final SchemaKStream result = outputNode.buildStream(ksqlStreamBuilder); // Then: verify(sourceStream).into( @@ -220,16 +230,11 @@ private void givenInsertIntoNode() { buildNode(); } - private void givenNodeWithSchema(final LogicalSchema schema) { - this.schema = schema; - buildNode(); - } - private void buildNode() { outputNode = new KsqlStructuredDataOutputNode( PLAN_NODE_ID, sourceNode, - schema, + SCHEMA, Optional.empty(), KEY_FIELD, ksqlTopic, @@ -238,4 +243,15 @@ private void buildNode() { SerdeOption.none(), SourceName.of(PLAN_NODE_ID.toString())); } + + private void givenSourceSelectExpressions(final SelectExpression... selectExpressions) { + when(sourceNode.getSelectExpressions()) + .thenReturn(ImmutableList.copyOf(selectExpressions)); + } + + private static SelectExpression selectExpression(final String alias) { + final SelectExpression mock = mock(SelectExpression.class); + when(mock.getAlias()).thenReturn(ColumnName.of(alias)); + return mock; + } } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java b/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java index 8c68954096c2..534a91da6d1a 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKTableTest.java @@ -91,6 +91,7 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.MetaStoreFixture; import io.confluent.ksql.util.Pair; +import io.confluent.ksql.util.SchemaUtil; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -121,7 +122,8 @@ @RunWith(MockitoJUnitRunner.class) public class SchemaKTableTest { - private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.STRING); + private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING); private final KsqlConfig ksqlConfig = new KsqlConfig(Collections.emptyMap()); private final MetaStore metaStore = MetaStoreFixture.getNewMetaStore(new InternalFunctionRegistry()); diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java index e2a0945ce860..c179fab7e981 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/ddl/commands/CreateSourceCommandTest.java @@ -68,7 +68,7 @@ public void shouldThrowOnMultipleKeyColumns() { } @Test - public void shouldThrowIfKeyFieldDoesNotMatchRowKeyType() { + public void shouldThrowIfKeyFieldDoesNotMatchKeyType() { // Given: final ColumnName keyField = ColumnName.of("keyField"); @@ -82,10 +82,10 @@ public void shouldThrowIfKeyFieldDoesNotMatchRowKeyType() { expectedException.expectMessage("The KEY field (keyField) identified in the " + "WITH clause is of a different type to the actual key column."); expectedException.expectMessage( - "Either change the type of the KEY field to match ROWKEY, or explicitly set ROWKEY " - + "to the type of the KEY field by adding 'ROWKEY STRING KEY' in the schema."); + "Use of the KEY field is deprecated. Remove the KEY field from the WITH clause and " + + "specify the name of the key column by adding 'keyField STRING KEY' to the schema."); expectedException.expectMessage("KEY field type: STRING"); - expectedException.expectMessage("ROWKEY type: INTEGER"); + expectedException.expectMessage("key column type: INTEGER"); // When: new TestCommand( @@ -100,7 +100,7 @@ public void shouldThrowIfKeyFieldDoesNotMatchRowKeyType() { } @Test - public void shouldNotThrowIfKeyFieldMatchesRowKeyType() { + public void shouldNotThrowIfKeyFieldMatchesKeyType() { // Given: final LogicalSchema schema = LogicalSchema.builder() .keyColumn(K0, SqlTypes.INTEGER) @@ -125,7 +125,7 @@ public void shouldNotThrowIfKeyFieldMatchesRowKeyType() { public void shouldThrowOnWindowStartColumn() { // Given: final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.INTEGER) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(SchemaUtil.WINDOWSTART_NAME, SqlTypes.INTEGER) .build(); @@ -149,7 +149,7 @@ public void shouldThrowOnWindowStartColumn() { public void shouldThrowOnWindowEndColumn() { // Given: final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.INTEGER) + .keyColumn(ColumnName.of("k1"), SqlTypes.INTEGER) .valueColumn(SchemaUtil.WINDOWEND_NAME, SqlTypes.INTEGER) .build(); diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/transform/select/SelectionTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/transform/select/SelectionTest.java index cb2a5c135391..c702e91e2fc4 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/transform/select/SelectionTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/transform/select/SelectionTest.java @@ -44,6 +44,7 @@ import org.mockito.junit.MockitoRule; public class SelectionTest { + private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) @@ -114,12 +115,12 @@ public void shouldBuildCorrectResultSchema() { final LogicalSchema resultSchema = selection.getSchema(); // Then: - final LogicalSchema expected = new LogicalSchema.Builder() + assertThat(resultSchema, equalTo(LogicalSchema.builder() .withRowTime() .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("FOO"), SqlTypes.STRING) .valueColumn(ColumnName.of("BAR"), SqlTypes.BIGINT) - .build(); - assertThat(resultSchema, equalTo(expected)); + .build() + )); } } \ No newline at end of file diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/util/StructKeyUtilTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/util/StructKeyUtilTest.java index 53910a87b48d..b3bec08b3c7a 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/util/StructKeyUtilTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/util/StructKeyUtilTest.java @@ -33,7 +33,7 @@ public class StructKeyUtilTest { private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("BOB"), SqlTypes.INTEGER) + .keyColumn(ColumnName.of("Bob"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("DOES_NOT_MATTER"), SqlTypes.STRING) .build(); @@ -61,7 +61,7 @@ public void shouldBuildCorrectSchema() { // Then: assertThat(result.schema(), is(SchemaBuilder.struct() - .field("ROWKEY", Schema.OPTIONAL_INT32_SCHEMA) + .field("Bob", Schema.OPTIONAL_INT32_SCHEMA) .build())); } @@ -71,7 +71,7 @@ public void shouldHandleValue() { final Struct result = builder.build(1); // Then: - assertThat(result.getInt32("ROWKEY"), is(1)); + assertThat(result.getInt32("Bob"), is(1)); } @Test @@ -80,6 +80,6 @@ public void shouldHandleNulls() { final Struct result = builder.build(null); // Then: - assertThat(result.getInt32("ROWKEY"), is(nullValue())); + assertThat(result.getInt32("Bob"), is(nullValue())); } } \ No newline at end of file diff --git a/ksql-functional-tests/README.md b/ksql-functional-tests/README.md index 0de71e4753c1..9705686aed7a 100644 --- a/ksql-functional-tests/README.md +++ b/ksql-functional-tests/README.md @@ -292,8 +292,7 @@ A post condition can define the list of sources that must exist in the metastore { "name": "S1", "type": "table", - "keyField": "FOO", - "valueSchema": "STRUCT" + "schema": "ID BIGINT KEY, FOO STRING, KSQL_COL_1 BIGINT" } ``` @@ -303,8 +302,7 @@ Each source can define the following attributes: |-------------|:------------| | name | (Required) The name of the source. | | type | (Required) Specifies if the source is a STREAM or TABLE. | -| keyField | (Optional) Specifies the keyField for the source. If present, but set to `null`, the name of the key field is expected to not be set. If not supplied, the name of the key field will not be checked. | -| valueSchema | (Optional) Specifies the value SQL schema for the source. | +| schema | (Optional) Specifies the SQL schema for the source. | #### Topics A post condition can define a check against the set of topics the case creates diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json b/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json index b659972e0e9f..ffebc694e47e 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/elements.json @@ -209,6 +209,34 @@ ] } }, + { + "name": "validate without value elements OK - custom key name", + "statements": [ + "CREATE STREAM INPUT (id int key) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM input;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "topics": [ + { + "name": "input", + "schema": {"type": "object","properties": {"c1": {"type": "integer"}}}, + "format": "JSON" + }, + { + "name": "OUTPUT", + "format": "JSON" + } + ], + "inputs": [{"topic": "input", "key": 42, "value": {"c1": 4}}], + "outputs": [{"topic": "OUTPUT", "key": 42, "value": {"C1": 4}}], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, C1 BIGINT"} + ] + } + }, { "name": "validate with elements OK", "format": ["JSON", "PROTOBUF"], @@ -574,10 +602,30 @@ "CREATE STREAM OUTPUT AS SELECT ROWKEY FROM INPUT;" ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Reserved column name in select: `ROWKEY`. Please remove or alias the column." } }, + { + "name": "non-join should reject KEY column name in projection", + "comments": [ + "changes to the ksql query semantics are required to allow this.", + "At the moment, the key schema passes through the select un-changed.", + "which means the key column in the projection is added to the value schema", + "but the name of the column clashes with the key column, resulting in an error" + ], + "statements": [ + "CREATE STREAM INPUT (K INT KEY, F0 INT) WITH (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT K FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Value column name(s) `K` clashes with key column name(s)." + } + }, { "name": "non-join should reject WINDOWSTART in projection", "statements": [ @@ -629,10 +677,31 @@ "CREATE STREAM OUTPUT as SELECT l.ROWKEY AS ROWKEY, f0, f1 FROM left_stream l join right_stream r WITHIN 1 seconds ON l.rowkey = r.rowkey;" ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Reserved column name in select: `ROWKEY`. Please remove or alias the column." } }, + { + "name": "join should reject KEY column name in projection", + "comments": [ + "changes to the ksql query semantics are required to allow this.", + "At the moment, the key schema passes through the select un-changed.", + "which means the key column in the projection is added to the value schema", + "but the name of the column clashes with the key column, resulting in an error" + ], + "statements": [ + "CREATE STREAM LEFT_STREAM (K DOUBLE KEY, F0 INT) WITH (kafka_topic='left', value_format='JSON');", + "CREATE STREAM RIGHT_STREAM (K DOUBLE KEY, F1 INT) WITH (kafka_topic='right', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT l.K AS K, f0, f1 FROM left_stream l join right_stream r WITHIN 1 seconds ON l.k = r.k;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Value column name(s) `K` clashes with key column name(s)." + } + }, { "name": "join should reject WINDOWSTART in projection", "statements": [ diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/group-by.json b/ksql-functional-tests/src/test/resources/query-validation-tests/group-by.json index d034e607f9fb..e4fe5ea5e304 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/group-by.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/group-by.json @@ -709,35 +709,43 @@ {"topic": "OUTPUT", "key": "1", "value": "2"}, {"topic": "OUTPUT", "key": "2", "value": "2"}, {"topic": "OUTPUT", "key": "1", "value": "3"} - ] + ], + "post": { + "topics": { + "blacklist": ".*-repartition" + } + } }, { - "name": "ROWKEY (stream->table) - without repartition", - "comment": [ - "Clone of test 'ROWKEY (stream->table)' but checking no repartition topic is created post v5.4" - ], + "name": "only key column (stream->table)", "statements": [ - "CREATE STREAM TEST (ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", - "CREATE TABLE OUTPUT AS SELECT COUNT(*) FROM TEST GROUP BY ROWKEY;" + "CREATE STREAM TEST (ID INT KEY, ignored VARCHAR) WITH (kafka_topic='test_topic', value_format='DELIMITED');", + "CREATE TABLE OUTPUT AS SELECT COUNT(*) FROM TEST GROUP BY ID;" ], + "properties": { + "ksql.any.key.name.enabled": true + }, "inputs": [ - {"topic": "test_topic", "key": "1", "value": "-"}, - {"topic": "test_topic", "key": "2", "value": "-"}, - {"topic": "test_topic", "key": "1", "value": "-"}, - {"topic": "test_topic", "key": "2", "value": "-"}, - {"topic": "test_topic", "key": "1", "value": "-"} + {"topic": "test_topic", "key": 1, "value": "-"}, + {"topic": "test_topic", "key": 2, "value": "-"}, + {"topic": "test_topic", "key": 1, "value": "-"}, + {"topic": "test_topic", "key": 2, "value": "-"}, + {"topic": "test_topic", "key": 1, "value": "-"} ], "outputs": [ - {"topic": "OUTPUT", "key": "1", "value": "1"}, - {"topic": "OUTPUT", "key": "2", "value": "1"}, - {"topic": "OUTPUT", "key": "1", "value": "2"}, - {"topic": "OUTPUT", "key": "2", "value": "2"}, - {"topic": "OUTPUT", "key": "1", "value": "3"} + {"topic": "OUTPUT", "key": 1, "value": "1"}, + {"topic": "OUTPUT", "key": 2, "value": "1"}, + {"topic": "OUTPUT", "key": 1, "value": "2"}, + {"topic": "OUTPUT", "key": 2, "value": "2"}, + {"topic": "OUTPUT", "key": 1, "value": "3"} ], "post": { "topics": { "blacklist": ".*-repartition" - } + }, + "sources": [ + {"name": "OUTPUT", "type": "table", "schema": "ID INT KEY, KSQL_COL_0 BIGINT"} + ] } }, { @@ -1424,7 +1432,73 @@ "name": "OUTPUT", "type": "table", "keyFormat": {"format": "KAFKA"}, - "schema": "`ROWKEY` INTEGER KEY, `KSQL_COL_0` BIGINT" + "schema": "ROWKEY INTEGER KEY, KSQL_COL_0 BIGINT" + } + ] + } + }, + { + "name": "only key column", + "statements": [ + "CREATE STREAM INPUT (ID INT KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE OUTPUT AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": 11, "value": {}}, + {"topic": "test_topic", "timestamp": 12365, "key": 10, "value": {}}, + {"topic": "test_topic", "timestamp": 12375, "key": 11, "value": {}} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12345, "key": 11, "value": {"KSQL_INTERNAL_COL_0": 11, "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12365, "key": 10, "value": {"KSQL_INTERNAL_COL_0": 10, "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12375, "key": 11, "value": {"KSQL_INTERNAL_COL_0": 11, "KSQL_AGG_VARIABLE_0": 2}}, + {"topic": "OUTPUT", "key": 11, "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": 10, "value": {"COUNT": 1}}, + {"topic": "OUTPUT", "key": 11, "value": {"COUNT": 2}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "table", + "keyFormat": {"format": "KAFKA"}, + "schema": "ID INT KEY, COUNT BIGINT" + } + ] + } + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM INPUT (`Key` STRING KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE OUTPUT AS SELECT COUNT(1) AS `Value` FROM INPUT GROUP BY `Key`;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12365, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 12375, "key": "11", "value": {}} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12345, "key": "11", "value": {"KSQL_INTERNAL_COL_0": "11", "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12365, "key": "10", "value": {"KSQL_INTERNAL_COL_0": "10", "KSQL_AGG_VARIABLE_0": 1}}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", "timestamp": 12375, "key": "11", "value": {"KSQL_INTERNAL_COL_0": "11", "KSQL_AGG_VARIABLE_0": 2}}, + {"topic": "OUTPUT", "key": "11", "value": {"Value": 1}}, + {"topic": "OUTPUT", "key": "10", "value": {"Value": 1}}, + {"topic": "OUTPUT", "key": "11", "value": {"Value": 2}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "table", + "keyFormat": {"format": "KAFKA"}, + "schema": "`Key` STRING KEY, `Value` BIGINT" } ] } diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json b/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json index 7ee7da78226a..533588519102 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/joins.json @@ -1,5 +1,1663 @@ { "tests": [ + { + "name": "stream-stream left join", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 0, "end": 11000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 0, "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 10000, "end": 21000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 10000, "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 11000, "end": 22000, "type": "time"}, "key": 10, "value": {"T_ROWTIME": 11000, "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 13000, "end": 24000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 13000, "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 15000, "end": 26000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 15000, "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 16000, "end": 27000, "type": "time"}, "key": 100, "value": {"TT_ROWTIME": 16000, "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 17000, "end": 28000, "type": "time"}, "key": 90, "value": {"T_ROWTIME": 17000, "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 30000, "end": 41000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 30000, "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "bar", "VALUE": 99, "F1": null, "F2": null}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream left join", + "format": ["PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 0, "end": 11000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 0, "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 10000, "end": 21000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 10000, "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 11000, "end": 22000, "type": "time"}, "key": 10, "value": {"T_ROWTIME": 11000, "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 13000, "end": 24000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 13000, "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 15000, "end": 26000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 15000, "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", "window": {"start": 16000, "end": 27000, "type": "time"}, "key": 100, "value": {"TT_ROWTIME": 16000, "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 17000, "end": 28000, "type": "time"}, "key": 90, "value": {"T_ROWTIME": 17000, "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 30000, "end": 41000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 30000, "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "bar", "VALUE": 99, "F1": "", "F2": 0}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream left join - KAFKA", + "statements": [ + "CREATE STREAM S_LEFT (ID BIGINT KEY, NAME varchar) WITH (kafka_topic='left_topic', value_format='KAFKA');", + "CREATE STREAM S_RIGHT (ID BIGINT KEY, NAME varchar) WITH (kafka_topic='right_topic', value_format='KAFKA');", + "CREATE STREAM OUTPUT WITH(value_format='delimited') as SELECT * FROM s_left join s_right WITHIN 1 second ON s_left.id = s_right.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Source(s) S_LEFT, S_RIGHT are using the 'KAFKA' value format. This format does not yet support JOIN." + } + }, + { + "name": "stream-stream left join with rowkey - rekey", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 10, "NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 0, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 100, "F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 90, "NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": null, "F2": null}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ROWKEY BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream left join with rowkey - rekey", + "format": ["PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, t.k, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 10, "NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 0, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": "foo", "value": {"ID": 100, "F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 90, "NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": "foo", "value": {"ID": 0, "NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "T_K": "foo","NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "T_K": "foo", "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "T_K": "foo", "NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "T_K": "foo", "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "T_K": "foo", "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "T_K": "foo", "NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "T_K": "foo", "NAME": "bar", "VALUE": 99, "F1": "", "F2": 0}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ROWKEY BIGINT KEY, T_ID BIGINT, T_K STRING, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream left join - rekey", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "value": {"ID": 0, "NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "value": {"ID": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "value": {"ID": 10, "NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "value": {"ID": 0, "NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "value": {"ID": 0, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "value": {"ID": 100, "F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "value": {"ID": 90, "NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "value": {"ID": 0, "NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 0, "T_K": "", "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 0, "value": {"TT_ROWTIME": 10000, "TT_K": "", "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 10, "value": {"T_ROWTIME": 11000, "T_K": "", "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 13000, "T_K": "", "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 0, "value": {"TT_ROWTIME": 15000, "TT_K": "", "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 100, "value": {"TT_ROWTIME": 16000, "TT_K": "", "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 90, "value": {"T_ROWTIME": 17000, "T_K": "", "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 30000, "T_K": "", "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 0, "end": 11000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 0, "T_K": "", "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 10000, "end": 21000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 10000, "TT_K": "", "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 11000, "end": 22000, "type": "time"}, "key": 10, "value": {"T_ROWTIME": 11000, "T_K": "", "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 13000, "end": 24000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 13000, "T_K": "", "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 15000, "end": 26000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 15000, "TT_K": "", "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 16000, "end": 27000, "type": "time"}, "key": 100, "value": {"TT_ROWTIME": 16000, "TT_K": "", "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 17000, "end": 28000, "type": "time"}, "key": 90, "value": {"T_ROWTIME": 17000, "T_K": "", "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 30000, "end": 41000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 30000, "T_K": "", "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": null, "F2": null}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ROWKEY BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream left join - rekey", + "format": ["PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "value": {"ID": 0, "NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "value": {"ID": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "value": {"ID": 10, "NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "value": {"ID": 0, "NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "value": {"ID": 0, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "value": {"ID": 100, "F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "value": {"ID": 90, "NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "value": {"ID": 0, "NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 0, "T_K": "", "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 0, "value": {"TT_ROWTIME": 10000, "TT_K": "", "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 10, "value": {"T_ROWTIME": 11000, "T_K": "", "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 13000, "T_K": "", "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 0, "value": {"TT_ROWTIME": 15000, "TT_K": "", "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", "key": 100, "value": {"TT_ROWTIME": 16000, "TT_K": "", "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 90, "value": {"T_ROWTIME": 17000, "T_K": "", "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", "key": 0, "value": {"T_ROWTIME": 30000, "T_K": "", "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 0, "end": 11000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 0, "T_K": "", "T_ID": 0, "T_NAME": "zero", "T_VALUE": 0}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 10000, "end": 21000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 10000, "TT_K": "", "TT_ID": 0, "TT_F1": "blah", "TT_F2": 50}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 11000, "end": 22000, "type": "time"}, "key": 10, "value": {"T_ROWTIME": 11000, "T_K": "", "T_ID": 10, "T_NAME": "100", "T_VALUE": 5}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 13000, "end": 24000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 13000, "T_K": "", "T_ID": 0, "T_NAME": "foo", "T_VALUE": 100}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 15000, "end": 26000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 15000, "TT_K": "", "TT_ID": 0, "TT_F1": "a", "TT_F2": 10}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", "window": {"start": 16000, "end": 27000, "type": "time"}, "key": 100, "value": {"TT_ROWTIME": 16000, "TT_K": "", "TT_ID": 100, "TT_F1": "newblah", "TT_F2": 150}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 17000, "end": 28000, "type": "time"}, "key": 90, "value": {"T_ROWTIME": 17000, "T_K": "", "T_ID": 90, "T_NAME": "ninety", "T_VALUE": 90}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", "window": {"start": 30000, "end": 41000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 30000, "T_K": "", "T_ID": 0, "T_NAME": "bar", "T_VALUE": 99}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "", "F2": 0}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ROWKEY BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream inner join", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN 11 SECONDS ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream inner join all left fields some right", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT t.*, tt.f1 FROM test t inner join TEST_STREAM tt WITHIN 11 SECONDS ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_NAME": "zero", "T_VALUE": 0, "F1": "blah", "T_ROWTIME": 0}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_NAME": "foo", "T_VALUE": 100, "F1": "blah", "T_ROWTIME": 13000}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_NAME": "foo", "T_VALUE": 100, "F1": "a", "T_ROWTIME": 13000}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, T_ROWTIME BIGINT, T_ID BIGINT, T_NAME STRING, T_VALUE BIGINT, F1 STRING"} + ] + } + }, + { + "name": "stream-stream inner join all right fields some left", + "format": ["AVRO", "JSON", "PROTOBUF"], + "properties": { + "ksql.any.key.name.enabled": true + }, + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT t.*, tt.name FROM test tt inner join TEST_STREAM t WITHIN 11 SECONDS ON t.id = tt.id;" + ], + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "blah", "T_F2": 50, "T_ROWTIME": 10000, "NAME": "zero"}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "blah", "T_F2": 50, "T_ROWTIME": 10000, "NAME": "foo"}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "a", "T_F2": 10, "T_ROWTIME": 15000, "NAME": "foo"}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, T_ROWTIME BIGINT, T_ID BIGINT, T_F1 STRING, T_F2 BIGINT, NAME STRING"} + ] + } + }, + { + "name": "stream-stream inner join all fields", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT * FROM test tt inner join TEST_STREAM t WITHIN 11 SECONDS ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero"}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah"}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100"}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo"}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a"}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah"}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety"}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar"}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 0, "end": 11000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 0, "TT_ID": 0, "TT_NAME": "zero"}, "timestamp": 0}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", "window": {"start": 10000, "end": 21000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 10000, "T_ID": 0, "T_F1": "blah"}, "timestamp": 10000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 11000, "end": 22000, "type": "time"}, "key": 10, "value": {"TT_ROWTIME": 11000, "TT_ID": 10, "TT_NAME": "100"}, "timestamp": 11000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 13000, "end": 24000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 13000, "TT_ID": 0, "TT_NAME": "foo"}, "timestamp": 13000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", "window": {"start": 15000, "end": 26000, "type": "time"}, "key": 0, "value": {"T_ROWTIME": 15000, "T_ID": 0, "T_F1": "a"}, "timestamp": 15000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", "window": {"start": 16000, "end": 27000, "type": "time"}, "key": 100, "value": {"T_ROWTIME": 16000, "T_ID": 100, "T_F1": "newblah"}, "timestamp": 16000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 17000, "end": 28000, "type": "time"}, "key": 90, "value": {"TT_ROWTIME": 17000, "TT_ID": 90, "TT_NAME": "ninety"}, "timestamp": 17000}, + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", "window": {"start": 30000, "end": 41000, "type": "time"}, "key": 0, "value": {"TT_ROWTIME": 30000, "TT_ID": 0, "TT_NAME": "bar"}, "timestamp": 30000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "blah", "T_ROWTIME": 10000, "TT_ID": 0, "TT_NAME": "zero", "TT_ROWTIME": 0}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "blah", "T_ROWTIME": 10000, "TT_ID": 0, "TT_NAME": "foo", "TT_ROWTIME": 13000}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "T_F1": "a", "T_ROWTIME": 15000, "TT_ID": 0, "TT_NAME": "foo", "TT_ROWTIME": 13000}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, TT_ROWTIME BIGINT, TT_ID BIGINT, TT_NAME STRING, T_ROWTIME BIGINT, T_ID BIGINT, T_F1 STRING"} + ] + } + }, + { + "name": "stream-stream inner join with different before and after windows", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN (11 seconds, 10 seconds) on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 11000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 12000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream inner join with out of order messages", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN 10 seconds on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 9999}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "late-message", "VALUE": 10000}, "timestamp": 6000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 9999}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "late-message", "VALUE": 10000, "F1": "blah", "F2": 50}, "timestamp": 9999}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "late-message", "VALUE": 10000, "F1": "a", "F2": 10}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream outer join", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_STREAM tt WITHIN 11 seconds on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 20000} + + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": null, "F2": null}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 100, "value": {"T_ID": null, "NAME": null, "VALUE": null, "F1": "newblah", "F2": 150}, "timestamp": 20000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "ID BIGINT KEY, T_ID BIGINT, NAME STRING, VALUE BIGINT, F1 STRING, F2 BIGINT"} + ] + } + }, + { + "name": "stream-stream outer join - PROTOBUF", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_STREAM tt WITHIN 11 seconds on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 30000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah", "F2": 150}, "timestamp": 20000} + + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "", "F2": 0}, "timestamp": 30000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 100, "value": {"T_ID": 0, "NAME": "", "VALUE": 0, "F1": "newblah", "F2": 150}, "timestamp": 20000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "table-table left join", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE TABLE LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_TABLE tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000} + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 16000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 17000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "table-table left join - PROTOBUF", + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE TABLE LEFT_OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_TABLE tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000} + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 16000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 17000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "table-table inner join", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE TABLE INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_TABLE tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 15, "value": {"F1": "c", "F2": 20}, "timestamp": 15500}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 16000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "table-table outer join", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE TABLE OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_TABLE tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 15, "value": {"F1": "c", "F2": 20}, "timestamp": 15500}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000} + ], + "outputs": [ + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": null, "F2": null}, "timestamp": 0}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": null, "F2": null}, "timestamp": 11000}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "OUTER_JOIN", "key": 15, "value": {"T_ID": null, "NAME": null, "VALUE": null, "F1": "c", "F2": 20}, "timestamp": 15500}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 16000} + ], + "post": { + "sources": [ + {"name": "OUTER_JOIN", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "table-table outer join - PROTOBUF", + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE TABLE OUTER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_TABLE tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100", "VALUE": 5}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "right_topic", "key": 15, "value": {"F1": "c", "F2": 20}, "timestamp": 15500}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000} + ], + "outputs": [ + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "", "F2": 0}, "timestamp": 0}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "zero", "VALUE": 0, "F1": "blah", "F2": 50}, "timestamp": 10000}, + {"topic": "OUTER_JOIN", "key": 10, "value": {"T_ID": 10, "NAME": "100", "VALUE": 5, "F1": "", "F2": 0}, "timestamp": 11000}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "blah", "F2": 50}, "timestamp": 13000}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "OUTER_JOIN", "key": 15, "value": {"T_ID": 0, "NAME": "", "VALUE": 0, "F1": "c", "F2": 20}, "timestamp": 15500}, + {"topic": "OUTER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 16000} + ], + "post": { + "sources": [ + {"name": "OUTER_JOIN", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "stream-table left join", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='test_table', value_format='{FORMAT}');", + "CREATE STREAM LEFT_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join test_table tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_table", "key": 0, "value": {"F1": "zero", "F2": 0}, "timestamp": 0}, + {"topic": "test_table", "key": 10, "value": {"F1": "100", "F2": 5}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "blah", "VALUE": 50}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 10000}, + {"topic": "test_table", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 15000}, + {"topic": "test_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 15000} + ], + "outputs": [ + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "blah", "VALUE": 50, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": null, "F2": null}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "LEFT_JOIN", "type": "stream", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "stream-table left join - PROTOBUF", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='test_table', value_format='PROTOBUF');", + "CREATE STREAM LEFT_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join test_table tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_table", "key": 0, "value": {"F1": "zero", "F2": 0}, "timestamp": 0}, + {"topic": "test_table", "key": 10, "value": {"F1": "100", "F2": 5}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "blah", "VALUE": 50}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 10000}, + {"topic": "test_table", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 15000}, + {"topic": "test_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 15000} + ], + "outputs": [ + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "blah", "VALUE": 50, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "LEFT_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "LEFT_JOIN", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "VALUE": 90, "F1": "", "F2": 0}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "LEFT_JOIN", "type": "stream", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "stream-table inner join", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='test_table', value_format='{FORMAT}');", + "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join test_table tt on t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_table", "key": 0, "value": {"F1": "zero", "F2": 0}, "timestamp": 0}, + {"topic": "test_table", "key": 10, "value": {"F1": "100", "F2": 5}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "blah", "VALUE": 50}, "timestamp": 10000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 10000}, + {"topic": "test_table", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "test_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 15000}, + {"topic": "test_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 15000} + ], + "outputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "blah", "VALUE": 50, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "VALUE": 100, "F1": "zero", "F2": 0}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "VALUE": 99, "F1": "a", "F2": 10}, "timestamp": 15000} + ], + "post": { + "sources": [ + {"name": "INNER_JOIN", "type": "stream", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "a table join pipeline", + "format": ["JSON"], + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE_2 (ID BIGINT KEY, F3 varchar) WITH (kafka_topic='right_topic_2', value_format='{FORMAT}');", + "CREATE TABLE INNER_JOIN WITH (PARTITIONS=4) as SELECT name, value, f1, f2 FROM test t join TEST_TABLE tt on t.id = tt.id;", + "CREATE TABLE INNER_JOIN_2 AS SELECT name, f1, f3 FROM inner_join tt join TEST_TABLE_2 t ON t.id = tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "topics": [ + { + "name": "INNER_JOIN", + "format": "JSON", + "partitions": 4 + } + ], + "inputs": [ + {"topic": "INNER_JOIN", "key": 0, "value": {"NAME": "X", "VALUE": 0, "F1": "yo dawg", "F2": 50}, "timestamp": 0}, + {"topic": "right_topic_2", "key": 0, "value": {"F3": "I heard you like joins"}, "timestamp": 10000}, + {"topic": "INNER_JOIN", "key": 100, "value": {"NAME": "X", "VALUE": 0, "F1": "KSQL has table-table joins", "F2": 50}, "timestamp": 15000}, + {"topic": "right_topic_2", "key": 100, "value": {"F3": "so now you can join your join"}, "timestamp": 20000} + ], + "outputs": [ + {"topic": "INNER_JOIN_2", "key": 0, "value": {"NAME": "X", "F1": "yo dawg", "F3": "I heard you like joins"}, "timestamp": 10000}, + {"topic": "INNER_JOIN_2", "key": 100, "value": {"NAME": "X", "F1": "KSQL has table-table joins", "F3": "so now you can join your join"}, "timestamp": 20000} + ] + }, + { + "name": "table-table join with where clause", + "statements": [ + "CREATE TABLE TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE TABLE OUTPUT as SELECT t.id, name, tt.f1, f2 FROM test t JOIN test_table tt ON t.id = tt.id WHERE t.value > 10 AND tt.f2 > 5;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero", "VALUE": 0}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah", "F2": 4}, "timestamp": 10000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo", "VALUE": 100}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar", "VALUE": 99}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety", "VALUE": 90}, "timestamp": 17000}, + {"topic": "right_topic", "key": 90, "value": {"F1": "b", "F2": 10}, "timestamp": 18000}, + {"topic": "right_topic", "key": 90, "value": null, "timestamp": 19000} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": null, "timestamp": 10000}, + {"topic": "OUTPUT", "key": 0, "value": null, "timestamp": 13000}, + {"topic": "OUTPUT", "key": 0, "value": {"T_ID": 0, "NAME": "foo", "F1": "a", "F2": 10}, "timestamp": 15000}, + {"topic": "OUTPUT", "key": 0, "value": {"T_ID": 0, "NAME": "bar", "F1": "a", "F2": 10}, "timestamp": 16000}, + {"topic": "OUTPUT", "key": 90, "value": {"T_ID": 90, "NAME": "ninety", "F1": "b", "F2": 10}, "timestamp": 18000}, + {"topic": "OUTPUT", "key": 90, "value": null, "timestamp": 19000} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "table", "schema": "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `F1` STRING, `F2` BIGINT"} + ] + } + }, + { + "name": "to table using something other than key column", + "statements": [ + "CREATE STREAM S (ID bigint) WITH (kafka_topic='S', value_format='JSON');", + "CREATE TABLE NO_KEY (ID bigint, NAME string) WITH (kafka_topic='NO_KEY', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT s.id, name FROM S JOIN NO_KEY t ON s.id = t.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column ROWKEY instead of ID" + } + }, + { + "name": "stream-stream wrapped single field value schema on inputs", + "statements": [ + "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='S1', value_format='JSON');", + "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='S2', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT s1.name name1, s2.name name2 FROM S1 JOIN S2 WITHIN 1 second ON s1.id = s2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "S1", "key": 0, "value": {"NAME": "a"}, "timestamp": 0}, + {"topic": "S2", "key": 0, "value": {"NAME": "b"}, "timestamp": 10}, + {"topic": "S1", "key": 0, "value": {"ID": null}, "timestamp": 20}, + {"topic": "S2", "key": 0, "value": {"ID": null}, "timestamp": 30} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": "b"}, "timestamp": 20}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": null}, "timestamp": 30}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": null}, "timestamp": 30} + ] + }, + { + "name": "stream-stream unwrapped single field value schema on inputs", + "issues": [ + "With the current implementation the null values are ignored by KS.", + "This is probably not what we want. We could treat null values as a null ID for streams.", + "Though this would not make sense for tables, where null is a tombstone" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "statements": [ + "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S1', value_format='JSON');", + "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S2', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT s1.name name1, s2.name name2 FROM S1 JOIN S2 WITHIN 1 second ON s1.id = s2.id;" + ], + "inputs": [ + {"topic": "S1", "key": 0, "value": "a", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "b", "timestamp": 10}, + {"topic": "S1", "key": 0, "value": null, "timestamp": 20}, + {"topic": "S2", "key": 0, "value": null, "timestamp": 30} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10} + ] + }, + { + "name": "stream-stream unwrapped single field value schema on inputs and output", + "issues": [ + "With the current implementation the null values are ignored by KS.", + "This is probably not what we want. We could treat null values as a null ID for streams.", + "Though this would not make sense for tables, where null is a tombstone" + ], + "statements": [ + "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S1', value_format='JSON');", + "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S2', value_format='JSON');", + "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT s1.name name FROM S1 JOIN S2 WITHIN 1 second ON s1.id = s2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "S1", "key": 0, "value": "a", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "b", "timestamp": 10}, + {"topic": "S1", "key": 0, "value": null, "timestamp": 20}, + {"topic": "S2", "key": 0, "value": null, "timestamp": 30} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": "a", "timestamp": 10} + ] + }, + { + "name": "stream-table wrapped single field value schema on inputs", + "statements": [ + "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='S', value_format='JSON');", + "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='T', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT s.name name1, t.name name2 FROM S JOIN T ON S.id = T.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T", "key": 0, "value": {"NAME": "b"}, "timestamp": 0}, + {"topic": "S", "key": 0, "value": {"NAME": "a"}, "timestamp": 10}, + {"topic": "S", "key": 0, "value": {"NAME": null}, "timestamp": 20}, + {"topic": "T", "key": 0, "value": {"NAME": null}, "timestamp": 30}, + {"topic": "S", "key": 0, "value": {"NAME": null}, "timestamp": 40}, + {"topic": "T", "key": 0, "value": null, "timestamp": 50}, + {"topic": "S", "key": 0, "value": {"NAME": "a"}, "timestamp": 60} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": "b"}, "timestamp": 20}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": null}, "timestamp": 40} + ] + }, + { + "name": "stream-table unwrapped single field value schema on inputs", + "issues": [ + "With the current implementation the null values of the stream are ignored by KS.", + "This is probably not what we want. We could treat null values as a null ID for streams.", + "Though this would not make sense for tables, where null is a tombstone" + ], + "statements": [ + "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S', value_format='JSON');", + "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT s.name name1, t.name name2 FROM S JOIN T ON S.id = T.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T", "key": 0, "value": "b", "timestamp": 0}, + {"topic": "S", "key": 0, "value": "a", "timestamp": 10}, + {"topic": "S", "key": 0, "value": null, "timestamp": 20}, + {"topic": "T", "key": 0, "value": null, "timestamp": 30}, + {"topic": "S", "key": 0, "value": null, "timestamp": 40} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10} + ] + }, + { + "name": "stream-table unwrapped single field value schema on inputs and output", + "issues": [ + "With the current implementation the null values of the stream are ignored by KS.", + "This is probably not what we want. We could treat null values as a null ID for streams.", + "Though this would not make sense for tables, where null is a tombstone." + ], + "statements": [ + "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='S', value_format='JSON');", + "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T', value_format='JSON');", + "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT s.name name FROM S JOIN T ON S.id = T.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T", "key": 0, "value": "b", "timestamp": 0}, + {"topic": "S", "key": 0, "value": "a", "timestamp": 10}, + {"topic": "S", "key": 0, "value": null, "timestamp": 20}, + {"topic": "T", "key": 0, "value": null, "timestamp": 30}, + {"topic": "S", "key": 0, "value": null, "timestamp": 40} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": "a", "timestamp": 10} + ] + }, + { + "name": "table-table wrapped single field value schema on inputs", + "statements": [ + "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='T1', value_format='JSON');", + "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='T2', value_format='JSON');", + "CREATE TABLE OUTPUT as SELECT t1.name name1, t2.name name2 FROM T1 JOIN T2 ON T1.id = T2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T1", "key": 0, "value": {"NAME": "a"}, "timestamp": 0}, + {"topic": "T2", "key": 0, "value": {"NAME": "b"}, "timestamp": 10}, + {"topic": "T1", "key": 0, "value": {"NAME": null}, "timestamp": 20}, + {"topic": "T2", "key": 0, "value": {"NAME": null}, "timestamp": 30}, + {"topic": "T1", "key": 0, "value": {"NAME": null}, "timestamp": 40}, + {"topic": "T1", "key": 0, "value": null, "timestamp": 50}, + {"topic": "T2", "key": 0, "value": null, "timestamp": 60} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": "b"}, "timestamp": 20}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": null}, "timestamp": 30}, + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": null, "NAME2": null}, "timestamp": 40}, + {"topic": "OUTPUT", "key": 0, "value": null, "timestamp": 50} + ] + }, + { + "name": "table-table unwrapped single field value schema on inputs", + "statements": [ + "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T1', value_format='JSON');", + "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T2', value_format='JSON');", + "CREATE TABLE OUTPUT as SELECT t1.name name1, t2.name name2 FROM T1 JOIN T2 ON T1.id = T2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T1", "key": 0, "value": "a", "timestamp": 0}, + {"topic": "T2", "key": 0, "value": "b", "timestamp": 10}, + {"topic": "T1", "key": 0, "value": null, "timestamp": 20}, + {"topic": "T2", "key": 0, "value": null, "timestamp": 30}, + {"topic": "T1", "key": 0, "value": null, "timestamp": 40} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": {"NAME1": "a", "NAME2": "b"}, "timestamp": 10}, + {"topic": "OUTPUT", "key": 0, "value": null, "timestamp": 20} + ] + }, + { + "name": "table-table unwrapped single field value schema on inputs and output", + "statements": [ + "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T1', value_format='JSON');", + "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (WRAP_SINGLE_VALUE=false, kafka_topic='T2', value_format='JSON');", + "CREATE TABLE OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT t1.name name FROM T1 JOIN T2 ON T1.id = T2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "T1", "key": 0, "value": "a", "timestamp": 0}, + {"topic": "T2", "key": 0, "value": "b", "timestamp": 10}, + {"topic": "T1", "key": 0, "value": null, "timestamp": 20}, + {"topic": "T2", "key": 0, "value": null, "timestamp": 30}, + {"topic": "T1", "key": 0, "value": null, "timestamp": 40} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0, "value": "a", "timestamp": 10}, + {"topic": "OUTPUT", "key": 0, "value": null, "timestamp": 20} + ] + }, + { + "name": "stream-stream left join - invalid join field - contains literal", + "statements": [ + "CREATE STREAM TEST1 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT * FROM test1 t left join test2 tt ON t.id = 0;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Invalid comparison expression '0' in join '(T.ID = 0)'. Each side of the join comparision must contain references from exactly one source." + } + }, + { + "name": "stream-stream left join - invalid join field on lhs- contains literal", + "statements": [ + "CREATE STREAM TEST1 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID BIGINT KEY, NAME STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT * FROM test1 t left join test2 tt ON 0 = t.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Invalid comparison expression '0' in join '(0 = T.ID)'. Each side of the join comparision must contain references from exactly one source." + } + }, + { + "name": "stream-stream join - contains function", + "statements": [ + "CREATE STREAM TEST1 (ID varchar) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID varchar) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.ID FROM test1 t join test2 tt WITHIN 30 SECONDS ON t.id = SUBSTRING(tt.id, 2);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "foo", "value": {"id": "foo"}, "timestamp": 0}, + {"topic": "right_topic", "key": "!foo", "value": {"id": "!foo"}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "foo", "value": {"T_ID": "foo"}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains CAST", + "statements": [ + "CREATE STREAM TEST1 (ID bigint KEY, x bigint) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID int KEY, x int) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT t.x FROM test1 t JOIN test2 tt WITHIN 30 seconds ON t.id = CAST(tt.id AS BIGINT);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"x": 2}, "timestamp": 10}, + {"topic": "right_topic", "key": 1, "value": {"x": 3}, "timestamp": 10} + ], + "outputs": [ + {"topic": "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", "key": 1, "value": {"TT_X": 3, "TT_ROWTIME": 10, "TT_ID": 1}, "timestamp": 10}, + {"topic": "OUTPUT", "key": 1, "value": {"T_X": 2}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains CAST double to int", + "statements": [ + "CREATE STREAM L (ID INT KEY, x bigint) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM R (ID DOUBLE KEY, x bigint) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT L.x FROM L JOIN R WITHIN 30 seconds ON L.id = CAST(R.id AS INT);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"x": 2}, "timestamp": 10}, + {"topic": "right_topic", "key": 1.0, "value": {"x": 3}, "timestamp": 11} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"L_X": 2}, "timestamp": 11} + ] + }, + { + "name": "stream-stream join - contains subscript", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, NAME STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (K STRING KEY, ID ARRAY) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.ID FROM test1 t JOIN test2 tt WITHIN 30 SECONDS ON t.id = tt.id[1];" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"name": "-"}, "timestamp": 0}, + {"topic": "left_topic", "key": 2, "value": {"name": "-"}, "timestamp": 5}, + {"topic": "right_topic", "key": "k", "value": {"id": [1,2,3]}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"T_ID": 1}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains arithmetic binary expression", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, NAME STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, NAME STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.ID, TT.ID FROM test1 t join test2 tt WITHIN 30 seconds ON t.id = tt.id + 1;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"name": "-"}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"name": "-"}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"T_ID": 1, "TT_ID": 0}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains arithmetic unary expression", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, NAME STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, NAME STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.NAME, TT.NAME FROM test1 t join test2 tt WITHIN 30 seconds ON t.id = -tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"name": "a"}, "timestamp": 0}, + {"topic": "right_topic", "key": -1, "value": {"name": "b"}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"T_NAME": "a", "TT_NAME": "b"}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains CASE expression", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.ID FROM test1 t join test2 tt WITHIN 30 SECONDS ON t.id = (CASE WHEN tt.id = 2 THEN 1 ELSE 3 END);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {}, "timestamp": 0}, + {"topic": "left_topic", "key": 3, "value": {}, "timestamp": 5}, + {"topic": "right_topic", "key": 2, "value": {}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"T_ID": 1}, "timestamp": 10} + ] + }, + { + "name": "stream-stream join - contains arithmetic unary expression flipped sides", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT T.ID FROM test1 t join test2 tt WITHIN 30 seconds ON -tt.id = t.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {}, "timestamp": 0}, + {"topic": "right_topic", "key": -1, "value": {}, "timestamp": 10} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"T_ID": 1}, "timestamp": 10} + ] + }, + { + "name": "stream-stream left join - invalid left join expression - field does not exist", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT * FROM test1 t left join test2 tt ON t.iid= tt.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Column 'T.IID' cannot be resolved." + } + }, + { + "name": "stream-stream left join - invalid right join expression - field does not exist", + "statements": [ + "CREATE STREAM TEST1 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST2 (ID INT KEY, IGNORED STRING) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT * FROM test1 t left join test2 tt ON t.id= tt.iid;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Column 'TT.IID' cannot be resolved." + } + }, + { + "name": "on unqualified join criteria", + "statements": [ + "CREATE STREAM TEST (LEFT_ID BIGINT KEY, NAME varchar) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM TEST_STREAM (RIGHT_ID BIGINT KEY, F1 varchar) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM LEFT_OUTER_JOIN as SELECT name, f1 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON left_id = right_id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 0, "value": {"NAME": "zero"}, "timestamp": 0}, + {"topic": "right_topic", "key": 0, "value": {"F1": "blah"}, "timestamp": 10000}, + {"topic": "left_topic", "key": 10, "value": {"NAME": "100"}, "timestamp": 11000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "foo"}, "timestamp": 13000}, + {"topic": "right_topic", "key": 0, "value": {"F1": "a"}, "timestamp": 15000}, + {"topic": "right_topic", "key": 100, "value": {"F1": "newblah"}, "timestamp": 16000}, + {"topic": "left_topic", "key": 90, "value": {"NAME": "ninety"}, "timestamp": 17000}, + {"topic": "left_topic", "key": 0, "value": {"NAME": "bar"}, "timestamp": 30000} + ], + "outputs": [ + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "F1": null}, "timestamp": 0}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "zero", "F1": "blah"}, "timestamp": 10000}, + {"topic": "LEFT_OUTER_JOIN", "key": 10, "value": {"NAME": "100", "F1": null}, "timestamp": 11000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "F1": "blah"}, "timestamp": 13000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "foo", "F1": "a"}, "timestamp": 15000}, + {"topic": "LEFT_OUTER_JOIN", "key": 90, "value": {"NAME": "ninety", "F1": null}, "timestamp": 17000}, + {"topic": "LEFT_OUTER_JOIN", "key": 0, "value": {"NAME": "bar", "F1": null}, "timestamp": 30000} + ], + "post": { + "sources": [ + {"name": "LEFT_OUTER_JOIN", "type": "stream", "schema": "LEFT_ID BIGINT KEY, NAME STRING, F1 STRING"} + ] + } + }, + { + "name": "on non STRING value column", + "statements": [ + "CREATE STREAM INPUT_STREAM (K STRING KEY, SF BIGINT) WITH (kafka_topic='stream_topic', value_format='JSON');", + "CREATE TABLE INPUT_TABLE (ID BIGINT KEY, TF INT) WITH (kafka_topic='table_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT_STREAM S JOIN INPUT_TABLE T on S.SF = T.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "table_topic", "key": 26589, "value": {"TF": 1}, "timestamp": 0}, + {"topic": "stream_topic", "key": "a", "value": {"SF": 12589}, "timestamp": 100}, + {"topic": "table_topic", "key": 12589, "value": {"TF": 12}, "timestamp": 200}, + {"topic": "stream_topic", "key": "b", "value": {"SF": 12589}, "timestamp": 300} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 12589, "value": {"S_K": "b", "S_ROWTIME": 300, "S_SF": 12589, "T_ROWTIME": 300, "T_ID": 12589, "T_TF": 12}, "timestamp": 300} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "ROWKEY BIGINT KEY, S_ROWTIME BIGINT, S_K STRING, S_SF BIGINT, T_ROWTIME BIGINT, T_ID BIGINT, T_TF INT" + } + ] + } + }, + { + "name": "on non key table column", + "statements": [ + "CREATE STREAM INPUT_STREAM (ID BIGINT KEY, SF BIGINT) WITH (kafka_topic='stream_topic', value_format='JSON');", + "CREATE TABLE INPUT_TABLE (K BIGINT KEY, ID BIGINT, TF INT) WITH (kafka_topic='table_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT_STREAM S JOIN INPUT_TABLE T on S.ID = T.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column K instead of ID" + } + }, + { + "name": "on KAFKA INT column", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM L (ID STRING KEY, l0 INT, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM R (ID STRING KEY, r0 INT, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT as SELECT L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "a", "value": {"L0": 10, "L1": 1}, "timestamp": 0}, + {"topic": "right_topic", "key": "b" ,"value": {"R0": 10, "R1": 2}, "timestamp": 10000} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 10, "value": {"L_ID": "a", "L1": 1, "R1": 2}, "timestamp": 10000} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY INT KEY, L_ID STRING, L1 INT, R1 INT"} + ] + } + }, + { + "name": "on KAFKA BIGINT column", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM L (ID STRING KEY, l0 BIGINT, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM R (ID STRING KEY, r0 BIGINT, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT as SELECT L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "a", "value": {"L0": 1000000000, "L1": 1}, "timestamp": 0}, + {"topic": "right_topic", "key": "b" ,"value": {"R0": 1000000000, "R1": 2}, "timestamp": 10000} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1000000000, "value": {"L_ID": "a", "L1": 1, "R1": 2}, "timestamp": 10000} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY BIGINT KEY, L_ID STRING, L1 INT, R1 INT"} + ] + } + }, + { + "name": "on KAFKA DOUBLE column", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM L (ID STRING KEY, l0 DOUBLE, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM R (ID STRING KEY, r0 DOUBLE, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT as SELECT L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "a", "value": {"L0": 1.23, "L1": 1}, "timestamp": 0}, + {"topic": "right_topic", "key": "b" ,"value": {"R0": 1.23, "R1": 2}, "timestamp": 10000} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1.23, "value": {"L_ID": "a", "L1": 1, "R1": 2}, "timestamp": 10000} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY DOUBLE KEY, L_ID STRING, L1 INT, R1 INT"} + ] + } + }, + { + "name": "on KAFKA STRING column", + "format": ["AVRO", "JSON", "PROTOBUF"], + "statements": [ + "CREATE STREAM L (ID STRING KEY, l0 STRING, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM R (ID STRING KEY, r0 STRING, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT as SELECT L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": "a", "value": {"L0": "x", "L1": 1}, "timestamp": 0}, + {"topic": "right_topic", "key": "b" ,"value": {"R0": "x", "R1": 2}, "timestamp": 10000} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "x", "value": {"L_ID": "a", "L1": 1, "R1": 2}, "timestamp": 10000} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, L_ID STRING, L1 INT, R1 INT"} + ] + } + }, + { + "name": "self-join", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, ID bigint) WITH (kafka_topic='left_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT * FROM INPUT s1 JOIN INPUT s2 WITHIN 1 HOUR ON s1.id = s2.id;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Can not join 'INPUT' to 'INPUT': self joins are not yet supported." + } + }, + { + "name": "matching session windowed", + "comments": [ + "Note: the first record on the right topic intersects with the session on the right side, but no row is output as keys must", + "be an EXACT BINARY match" + ], + "statements": [ + "CREATE STREAM S1 (ID INT KEY, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM S2 (ID INT KEY, V bigint) WITH (kafka_topic='right_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM OUTPUT as SELECT S1.V, S2.V FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"V": 1}, "timestamp": 765, "window": {"start": 234, "end": 765, "type": "session"}}, + {"topic": "right_topic", "key": 1, "value": {"V": 2}, "timestamp": 567, "window": {"start": 234, "end": 567, "type": "session"}}, + {"topic": "right_topic", "key": 1, "value": {"V": 3}, "timestamp": 765, "window": {"start": 234, "end": 765, "type": "session"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"S1_V": 1, "S2_V": 3}, "timestamp": 765, "window": {"start": 234, "end": 765, "type": "session"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA", "windowType": "SESSION"}, + "schema": "ID INT KEY, S1_V BIGINT, S2_V BIGINT" + } + ] + } + }, + { + "name": "matching time windowed", + "comments": [ + "Note: the two streams use a different window size. However, only the start of the window is serialized, so its possible to get a matching binary key", + "This may meet users requirements, hence KSQL allows such joins", + "Note: the key format is currently taken from the left source." + ], + "statements": [ + "CREATE STREAM S1 (ID INT KEY, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='Hopping', WINDOW_SIZE='5 SECONDS');", + "CREATE STREAM S2 (ID INT KEY, V bigint) WITH (kafka_topic='right_topic', value_format='JSON', WINDOW_TYPE='Tumbling', WINDOW_SIZE='2 SECOND');", + "CREATE STREAM OUTPUT as SELECT * FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "left_topic", "key": 1, "value": {"V": 1}, "timestamp": 0, "window": {"start": 0, "end": 5000, "type": "time"}}, + {"topic": "left_topic", "key": 1, "value": {"V": 2}, "timestamp": 1000, "window": {"start": 1000, "end": 6000, "type": "time"}}, + {"topic": "left_topic", "key": 1, "value": {"V": 3}, "timestamp": 2000, "window": {"start": 2000, "end": 7000, "type": "time"}}, + {"topic": "right_topic", "key": 1, "value": {"V": 4}, "timestamp": 0, "window": {"start": 0, "end": 2000, "type": "time"}}, + {"topic": "right_topic", "key": 1, "value": {"V": 5}, "timestamp": 2000, "window": {"start": 2000, "end": 4000, "type": "time"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 1, "value": {"S1_ROWTIME": 0, "S1_WINDOWSTART": 0, "S1_WINDOWEND": 5000, "S1_ID": 1, "S1_V": 1, "S2_ROWTIME": 0, "S2_WINDOWSTART": 0, "S2_WINDOWEND": 2000, "S2_ID": 1, "S2_V": 4}, "timestamp": 0, "window": {"start": 0, "end":5000, "type": "time"}}, + {"topic": "OUTPUT", "key": 1, "value": {"S1_ROWTIME": 2000, "S1_WINDOWSTART": 2000, "S1_WINDOWEND": 7000, "S1_ID": 1, "S1_V": 3, "S2_ROWTIME": 2000, "S2_WINDOWSTART": 2000, "S2_WINDOWEND": 4000, "S2_ID": 1, "S2_V": 5}, "timestamp": 2000, "window": {"start": 2000, "end":7000, "type": "time"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA", "windowType": "HOPPING", "windowSize": 5000}, + "schema": "`ID` INTEGER KEY, `S1_ROWTIME` BIGINT, `S1_ID` INTEGER, `S1_WINDOWSTART` BIGINT, `S1_WINDOWEND` BIGINT, `S1_V` BIGINT, `S2_ROWTIME` BIGINT, `S2_ID` INTEGER, `S2_WINDOWSTART` BIGINT, `S2_WINDOWEND` BIGINT, `S2_V` BIGINT" + } + ] + } + }, + { + "name": "session and timed windowed", + "comments": [ + "Session windows serialize both start and end window bounds, where as tumbling/hopping only serialize the start time.", + "Keys will never be binary compatible, and hence KSQL should disallow such joins" + ], + "statements": [ + "CREATE STREAM S1 (ID INT KEY, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='Session');", + "CREATE STREAM S2 (ID INT KEY, V bigint) WITH (kafka_topic='right_topic', value_format='JSON', WINDOW_TYPE='TUMBLING', WINDOW_SIZE='1 SECOND');", + "CREATE STREAM OUTPUT as SELECT * FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Incompatible windowed sources.\nLeft source: SESSION\nRight source: TUMBLING\nSession windowed sources can only be joined to other session windowed sources, and may still not result in expected behaviour as session bounds must be an exact match for the join to work\nHopping and tumbling windowed sources can only be joined to other hopping and tumbling windowed sources" + } + }, + { + "name": "windowed vand non-windowed - INT", + "statements": [ + "CREATE STREAM S1 (ID INT KEY, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM S2 (ID INT KEY, V bigint) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT * FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Can not join windowed source to non-windowed source.\n`S1` is SESSION windowed\n`S2` is not windowed" + } + }, + { + "name": "windowed and non-windowed - STRING", + "statements": [ + "CREATE STREAM S1 (ID STRING KEY, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM S2 (ID STRING KEY, V bigint) WITH (kafka_topic='right_topic', value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT * FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Can not join windowed source to non-windowed source.\n`S1` is SESSION windowed\n`S2` is not windowed" + } + }, + { + "name": "requiring repartition of windowed source", + "statements": [ + "CREATE STREAM S1 (K INT KEY, ID INT, V bigint) WITH (kafka_topic='left_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM S2 (K INT KEY, ID INT, V bigint) WITH (kafka_topic='right_topic', value_format='JSON', WINDOW_TYPE='SESSION');", + "CREATE STREAM OUTPUT as SELECT * FROM S1 JOIN S2 WITHIN 1 MINUTE ON S1.ID = S2.ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Implicit repartitioning of windowed sources is not supported. See https://github.com/confluentinc/ksql/issues/4385." + } + }, + + + + + { "name": "stream stream left join", "format": ["AVRO", "JSON"], @@ -1531,7 +3189,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE key ROWKEY instead of ID" + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column ROWKEY instead of ID" } }, { @@ -2032,7 +3690,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE key ROWKEY instead of ID" + "message": "Cannot repartition a TABLE source. If this is a join, make sure that the criteria uses the TABLE's key column ROWKEY instead of ID" } }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json b/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json index c631d12c8b3e..a34e4b6b1c9c 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/key-field.json @@ -831,7 +831,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "The KEY field (FOO) identified in the WITH clause is of a different type to the actual key column.\nEither change the type of the KEY field to match ROWKEY, or explicitly set ROWKEY to the type of the KEY field by adding 'ROWKEY INTEGER KEY' in the schema.\nKEY field type: INTEGER\nROWKEY type: STRING" + "message": "The KEY field (FOO) identified in the WITH clause is of a different type to the actual key column.\nUse of the KEY field is deprecated. Remove the KEY field from the WITH clause and specify the name of the key column by adding 'FOO INTEGER KEY' to the schema.\nKEY field type: INTEGER\nkey column type: STRING" } }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json b/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json index 2acb23c20dbe..a8e5ce7a6864 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/key-schemas.json @@ -318,6 +318,59 @@ }, { "name": "explicit key field named other than ROWKEY", + "statements": [ + "CREATE STREAM INPUT (OTHER DOUBLE KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT ID, OTHER as KEY FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "input", "key": 3.0, "value": {"id": 1}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 3.0, "value": {"ID": 1, "KEY": 3.0}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "OTHER DOUBLE KEY, ID BIGINT, KEY DOUBLE" + } + ] + } + }, + { + "name": "KEY key field name", + "comment": "tests that `KEY` is allowed as a KEY column name", + "statements": [ + "CREATE STREAM INPUT (KEY STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');", + "CREATE STREAM OUTPUT as SELECT ID, KEY as KEY2 FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "input", "key": "a", "value": {"id": 1}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "a", "value": {"ID": 1, "KEY2": "a"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "KEY STRING KEY, ID BIGINT, KEY2 STRING" + } + ] + } + }, + { + "name": "explicit key field named other than ROWKEY - old", "statements": [ "CREATE STREAM INPUT (OTHER STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');" ], @@ -327,7 +380,7 @@ } }, { - "name": "KEY key field name", + "name": "KEY key field name- old", "comment": "tests that `KEY` is allowed as a KEY column name", "statements": [ "CREATE STREAM INPUT (KEY STRING KEY, ID bigint) WITH (kafka_topic='input',value_format='JSON');" diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/partition-by.json b/ksql-functional-tests/src/test/resources/query-validation-tests/partition-by.json index cad4ee7b0fa5..5879cef22ba7 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/partition-by.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/partition-by.json @@ -187,7 +187,30 @@ "post": { "topics": { "blacklist": ".*-repartition" - } + }, + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ROWKEY STRING KEY, ID BIGINT"} + ] + } + }, + { + "name": "partition by only key column", + "statements": [ + "CREATE STREAM INPUT (ID INT KEY, NAME STRING) with (kafka_topic='input', value_format='JSON');", + "CREATE STREAM OUTPUT AS select * from INPUT partition by ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [{"topic": "input", "key": 10, "value": {"NAME": "bob"}}], + "outputs": [{"topic": "OUTPUT", "key": 10, "value": {"NAME": "bob"}}], + "post": { + "topics": { + "blacklist": ".*-repartition" + }, + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] } }, { @@ -304,6 +327,66 @@ {"name": "OUTPUT", "type": "stream", "keyField": null} ] } + }, + { + "name": "only key column", + "statements": [ + "CREATE STREAM INPUT (ID INT KEY, NAME STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT PARTITION BY ID;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": 11, "value": {"name": "a"}}, + {"topic": "test_topic", "timestamp": 12365, "key": 10, "value": {"name": "b"}}, + {"topic": "test_topic", "timestamp": 12375, "key": 11, "value": {"name": "c"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 11, "value": {"NAME": "a"}}, + {"topic": "OUTPUT", "key": 10, "value": {"NAME": "b"}}, + {"topic": "OUTPUT", "key": 11, "value": {"NAME": "c"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "ID INT KEY, NAME STRING" + } + ] + } + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM INPUT (`Key` STRING KEY, `Name` STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT *, `Name` AS `Name2` FROM INPUT PARTITION BY `Key`;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "x", "value": {"Name": "a"}}, + {"topic": "test_topic", "timestamp": 12365, "key": "y", "value": {"Name": "b"}}, + {"topic": "test_topic", "timestamp": 12375, "key": "x", "value": {"Name": "c"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "x", "value": {"Name": "a", "Name2": "a"}}, + {"topic": "OUTPUT", "key": "y", "value": {"Name": "b", "Name2": "b"}}, + {"topic": "OUTPUT", "key": "x", "value": {"Name": "c", "Name2": "c"}} + ], + "post": { + "sources": [ + { + "name": "OUTPUT", + "type": "stream", + "keyFormat": {"format": "KAFKA"}, + "schema": "`Key` STRING KEY, `Name` STRING, `Name2` STRING" + } + ] + } } ] } diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/select.json b/ksql-functional-tests/src/test/resources/query-validation-tests/select.json new file mode 100644 index 000000000000..6add197aacbf --- /dev/null +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/select.json @@ -0,0 +1,91 @@ +{ + "comments": [ + "Tests covering general SELECT clause, a.k.a projection, handling" + ], + "tests": [ + { + "name": "key column", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT id as ID_COPY FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 8, "value": {"name": "a"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 8, "value": {"ID_COPY": 8}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, ID_COPY INT"} + ] + } + }, + { + "name": "value column", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT NAME FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 8, "value": {"name": "a"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 8, "value": {"NAME": "a"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] + } + }, + { + "name": "all columns - star", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 8, "value": {"name": "a"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 8, "value": {"NAME": "a"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] + } + }, + { + "name": "all columns - explicit", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID AS ID_COPY, ROWTIME AS ROWTIME_COPY, NAME FROM INPUT;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 1234, "key": 8, "value": {"name": "a"}} + ], + "outputs": [ + {"topic": "OUTPUT", "timestamp": 1234, "key": 8, "value": {"ID_COPY": 8, "ROWTIME_COPY": 1234, "NAME": "a"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, ID_COPY INT, ROWTIME_COPY BIGINT, NAME STRING"} + ] + } + } + ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json b/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json index cd832344495e..ac4a72dfa126 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/system-columns.json @@ -4,7 +4,17 @@ ], "tests": [ { - "name": "should fail if ROWKEY used as column name", + "name": "should fail if ROWKEY used as key column name", + "statements": [ + "CREATE STREAM INPUT (ROWKEY int) WITH (kafka_topic='test', value_format='JSON');" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "'ROWKEY' is a reserved column name. It can only be used for KEY columns." + } + }, + { + "name": "should fail if ROWKEY used as value column name", "statements": [ "CREATE STREAM INPUT (x int) WITH (kafka_topic='test', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT x AS rowkey FROM INPUT;" diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/table-functions.json b/ksql-functional-tests/src/test/resources/query-validation-tests/table-functions.json index cddb568aab47..4f17a0e21700 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/table-functions.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/table-functions.json @@ -152,6 +152,27 @@ {"topic": "OUTPUT", "key": "0", "value": {"KSQL_COL_0": "Struct{A=bar}"}} ] }, + { + "name": "test_udtf - key column", + "statements": [ + "CREATE STREAM TEST (ID INT KEY, NAME STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT TEST_UDTF(ID) FROM TEST;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 10, "value": {"NAME": "bob"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 10, "value": {"KSQL_COL_0": 10}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, KSQL_COL_0 INT"} + ] + } + }, { "name": "test_udtf - array params", "statements": [ diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/where.json b/ksql-functional-tests/src/test/resources/query-validation-tests/where.json new file mode 100644 index 000000000000..53cbaffb10e0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/where.json @@ -0,0 +1,57 @@ +{ + "comments": [ + "Tests covering general WHERE clause handling" + ], + "tests": [ + { + "name": "on key column", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT WHERE id < 10;" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 8, "value": {"name": "a"}}, + {"topic": "test_topic", "key": 9, "value": {"name": "a"}}, + {"topic": "test_topic", "key": 10, "value": {"name": "b"}}, + {"topic": "test_topic", "key": 11, "value": {"name": "c"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 8, "value": {"NAME": "a"}}, + {"topic": "OUTPUT", "key": 9, "value": {"NAME": "a"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] + } + }, + { + "name": "on value column", + "statements": [ + "CREATE STREAM INPUT (id int KEY, name STRING) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT WHERE name not like '%not%';" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "key": 8, "value": {"name": "this one"}}, + {"topic": "test_topic", "key": 9, "value": {"name": "not this one"}}, + {"topic": "test_topic", "key": 10, "value": {"name": "and this one"}}, + {"topic": "test_topic", "key": 11, "value": {"name": "but not this one"}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 8, "value": {"NAME": "this one"}}, + {"topic": "OUTPUT", "key": 10, "value": {"NAME": "and this one"}} + ], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "ID INT KEY, NAME STRING"} + ] + } + } + ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json index 8765665c1b53..bea23532e6fd 100644 --- a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json +++ b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json @@ -5,6 +5,402 @@ "tests": [ { "name": "explicitly supply all column values", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (ROWTIME, K, ID) VALUES (1234, 'key', 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": {"ID": 10}} + ], + "responses": [ + {"admin": {"@type": "currentStatus", "statementText": "{STATEMENT}"}} + ] + }, + { + "name": "explicitly supply values out of order", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (ID, ROWTIME, K) VALUES (10, 1234, 'key');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": {"ID": 10}} + ] + }, + { + "name": "explicitly supply default set of column values", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K, ID) VALUES ('key', 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "key", "value": {"ID": 10}} + ] + }, + { + "name": "implicitly supply default set of column values", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST VALUES ('key', 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "key", "value": {"ID": 10}} + ] + }, + { + "name": "should insert nulls for any fields not provided", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K) VALUES ('10');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "10", "value": {"ID": null}} + ] + }, + { + "name": "should insert null key", + "statements": [ + "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (ID) VALUES (10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"ID": 10}} + ] + }, + { + "name": "rowkey should be set when stream has int key and only key specified in insert", + "statements": [ + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (ID) VALUES (10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 10, "value": {"ID": 10}} + ] + }, + { + "name": "rowkey should be set when stream has String key and only key specified in insert", + "statements": [ + "CREATE STREAM TEST (ID VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (ID) VALUES ('10');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "10", "value": {"ID": "10"}} + ] + }, + { + "name": "rowkey should be set when stream has double key and only key specified in insert", + "statements": [ + "CREATE STREAM TEST (K DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (ID) VALUES (1.23);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 1.23, "value": {"ID": 1.23}} + ] + }, + { + "name": "rowkey should be set when stream has bigint key and only key specified in insert", + "statements": [ + "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (ID) VALUES (10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 10, "value": {"ID": 10}} + ] + }, + { + "name": "keyfield should be set when stream has string key and only rowkey specified in insert", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K) VALUES ('10');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "10", "value": {"ID": "10"}} + ] + }, + { + "name": "rowkey and key should match when stream has int key", + "statements": [ + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 10, "value": {"ID": 10}} + ] + }, + { + "name": "rowkey and key should match when stream has String key", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES ('10', '10');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "10", "value": {"ID": "10"}} + ] + }, + { + "name": "rowkey and key should match when stream has double key", + "statements": [ + "CREATE STREAM TEST (K DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (1.23, 1.23);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 1.23, "value": {"ID": 1.23}} + ] + }, + { + "name": "rowkey and key should match when stream has bigint key", + "statements": [ + "CREATE STREAM TEST (K BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": 10, "value": {"ID": 10}} + ] + }, + { + "name": "should fail on mismatch between explicit columns and value counts", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (K, ID) VALUES ('10');" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Failed to prepare statement: Expected number columns and values to match: [K, ID], ['10']", + "status": 400 + } + }, + { + "name": "should fail on mismatch between rowkey and key values when stream has key", + "statements": [ + "CREATE STREAM TEST (K INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "INSERT INTO TEST (K, ID) VALUES (10, 5);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Failed to insert values into 'TEST'. Expected K and ID to match but got 10 and 5 respectively.", + "status": 400 + } + }, + { + "name": "should coerce numbers", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, I INT, BI BIGINT, D DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (I, BI, D) VALUES (1, 2, 3);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"I": 1, "BI": 2, "D": 3.0}} + ] + }, + { + "name": "should handle arbitrary expressions", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, I INT, A ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (I, A) VALUES (-1, ARRAY[1, 1 + 1, 3]);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"I": -1, "A": [1, 2, 3]}} + ] + }, + { + "name": "should handle arbitrary nested expressions", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, I INT, A ARRAY>) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (I, A) VALUES (-1, ARRAY[ARRAY[1]]);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"I": -1, "A": [[1]]}} + ] + }, + { + "name": "should handle map expressions", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, I INT, A MAP) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (I, A) VALUES (-1, MAP('a':=0, 'b':=1));" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"I": -1, "A": {"a": 0, "b": 1}}} + ] + }, + { + "name": "should handle quoted identifiers", + "statements": [ + "CREATE STREAM `test` (`@key` STRING KEY, `id!` INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO `test` (ROWTIME, `@key`, `id!`) VALUES (1234, 'key', 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": {"id!": 10}} + ], + "responses": [ + {"admin": {"@type": "currentStatus", "statementText": "{STATEMENT}"}} + ] + }, + { + "name": "should handle struct expressions", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2.1, `bar` := ARRAY['bar']));" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"VAL": {"FOO": 2.1, "bar": ["bar"]}}} + ] + }, + { + "name": "should handle struct coercion", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, val STRUCT, baz DOUBLE>) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2, BAR := ARRAY[2], BAZ := 2));" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"VAL": {"FOO": 2, "BAR": [2], "BAZ": 2.0}}} + ] + }, + { + "name": "should handle empty struct expressions", + "statements": [ + "CREATE STREAM TEST (K STRING KEY, val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (val) VALUES (STRUCT());" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": null, "value": {"VAL": {"FOO": null, "bar": null}}} + ] + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM TEST (`Key` STRING KEY, `Value` INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "INSERT INTO TEST (`Key`, `Value`) VALUES ('key', 10);" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + ], + "outputs": [ + {"topic": "test_topic", "key": "key", "value": {"Value": 10}} + ] + }, + { + "name": "explicitly supply all column values - old`", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ROWTIME, ROWKEY, ID) VALUES (1234, 'key', 10);" @@ -19,7 +415,7 @@ ] }, { - "name": "explicitly supply values out of order", + "name": "explicitly supply values out of order - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ID, ROWTIME, ROWKEY) VALUES (10, 1234, 'key');" @@ -31,7 +427,7 @@ ] }, { - "name": "explicitly supply default set of column values", + "name": "explicitly supply default set of column values - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ROWKEY, ID) VALUES ('key', 10);" @@ -43,7 +439,7 @@ ] }, { - "name": "implicitly supply default set of column values", + "name": "implicitly supply default set of column values - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST VALUES ('key', 10);" @@ -55,7 +451,7 @@ ] }, { - "name": "should insert nulls for any fields not provided", + "name": "should insert nulls for any fields not provided - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ROWKEY) VALUES ('10');" @@ -67,7 +463,7 @@ ] }, { - "name": "should insert null key", + "name": "should insert null key - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ID) VALUES (10);" @@ -79,7 +475,7 @@ ] }, { - "name": "rowkey should be set when stream has int key and only key specified in insert", + "name": "rowkey should be set when stream has int key and only key specified in insert - old", "statements": [ "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (10);" @@ -91,7 +487,7 @@ ] }, { - "name": "rowkey should be set when stream has String key and only key specified in insert", + "name": "rowkey should be set when stream has String key and only key specified in insert - old", "statements": [ "CREATE STREAM TEST (ID VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES ('10');" @@ -103,7 +499,7 @@ ] }, { - "name": "rowkey should be set when stream has double key and only key specified in insert", + "name": "rowkey should be set when stream has double key and only key specified in insert - old", "statements": [ "CREATE STREAM TEST (ROWKEY DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (1.23);" @@ -115,7 +511,7 @@ ] }, { - "name": "rowkey should be set when stream has bigint key and only key specified in insert", + "name": "rowkey should be set when stream has bigint key and only key specified in insert - old", "statements": [ "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ID) VALUES (10);" @@ -127,7 +523,7 @@ ] }, { - "name": "keyfield should be set when stream has string key and only rowkey specified in insert", + "name": "keyfield should be set when stream has string key and only rowkey specified in insert - old", "statements": [ "CREATE STREAM TEST (ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY) VALUES ('10');" @@ -139,7 +535,7 @@ ] }, { - "name": "rowkey and key should match when stream has int key", + "name": "rowkey and key should match when stream has int key - old", "statements": [ "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 10);" @@ -151,7 +547,7 @@ ] }, { - "name": "rowkey and key should match when stream has String key", + "name": "rowkey and key should match when stream has String key - old", "statements": [ "CREATE STREAM TEST (ID STRING) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY, ID) VALUES ('10', '10');" @@ -163,7 +559,7 @@ ] }, { - "name": "rowkey and key should match when stream has double key", + "name": "rowkey and key should match when stream has double key - old", "statements": [ "CREATE STREAM TEST (ROWKEY DOUBLE KEY, ID DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY, ID) VALUES (1.23, 1.23);" @@ -175,7 +571,7 @@ ] }, { - "name": "rowkey and key should match when stream has bigint key", + "name": "rowkey and key should match when stream has bigint key - old", "statements": [ "CREATE STREAM TEST (ROWKEY BIGINT KEY, ID BIGINT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 10);" @@ -187,7 +583,7 @@ ] }, { - "name": "should fail on mismatch between explicit columns and value counts", + "name": "should fail on mismatch between explicit columns and value counts - old", "statements": [ "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (ROWKEY, ID) VALUES ('10');" @@ -199,7 +595,7 @@ } }, { - "name": "should fail on mismatch between rowkey and key values when stream has key", + "name": "should fail on mismatch between rowkey and key values when stream has key - old", "statements": [ "CREATE STREAM TEST (ROWKEY INT KEY, ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", "INSERT INTO TEST (ROWKEY, ID) VALUES (10, 5);" @@ -211,7 +607,7 @@ } }, { - "name": "should coerce numbers", + "name": "should coerce numbers - old", "statements": [ "CREATE STREAM TEST (I INT, BI BIGINT, D DOUBLE) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (I, BI, D) VALUES (1, 2, 3);" @@ -223,7 +619,7 @@ ] }, { - "name": "should handle arbitrary expressions", + "name": "should handle arbitrary expressions - old", "statements": [ "CREATE STREAM TEST (I INT, A ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (I, A) VALUES (-1, ARRAY[1, 1 + 1, 3]);" @@ -235,7 +631,7 @@ ] }, { - "name": "should handle arbitrary nested expressions", + "name": "should handle arbitrary nested expressions - old", "statements": [ "CREATE STREAM TEST (I INT, A ARRAY>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (I, A) VALUES (-1, ARRAY[ARRAY[1]]);" @@ -247,7 +643,7 @@ ] }, { - "name": "should handle map expressions", + "name": "should handle map expressions - old", "statements": [ "CREATE STREAM TEST (I INT, A MAP) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (I, A) VALUES (-1, MAP('a':=0, 'b':=1));" @@ -259,7 +655,7 @@ ] }, { - "name": "should handle quoted identifiers", + "name": "should handle quoted identifiers - old", "statements": [ "CREATE STREAM `test` (`id!` INT) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO `test` (ROWTIME, ROWKEY, `id!`) VALUES (1234, 'key', 10);" @@ -274,7 +670,7 @@ ] }, { - "name": "should handle struct expressions", + "name": "should handle struct expressions - old", "statements": [ "CREATE STREAM TEST (val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2.1, `bar` := ARRAY['bar']));" @@ -286,7 +682,7 @@ ] }, { - "name": "should handle struct coercion", + "name": "should handle struct coercion - old", "statements": [ "CREATE STREAM TEST (val STRUCT, baz DOUBLE>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT(FOO := 2, BAR := ARRAY[2], BAZ := 2));" @@ -298,7 +694,7 @@ ] }, { - "name": "should handle empty struct expressions", + "name": "should handle empty struct expressions - old", "statements": [ "CREATE STREAM TEST (val STRUCT>) WITH (kafka_topic='test_topic', value_format='JSON');", "INSERT INTO TEST (val) VALUES (STRUCT());" diff --git a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json index c33ae56576b0..d6af70e88c5f 100644 --- a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json +++ b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/pull-queries-against-materialized-aggregates.json @@ -105,7 +105,7 @@ ] }, { - "name": "non-windowed lookup on wrong type type", + "name": "non-windowed lookup on wrong type", "statements": [ "CREATE STREAM INPUT (ROWKEY INT KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", @@ -113,7 +113,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "'10' can not be converted to the type of column ROWKEY: INTEGER", + "message": "'10' can not be converted to the type of the key column: ROWKEY INTEGER KEY", "status": 400 } }, @@ -222,7 +222,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "'10' can not be converted to the type of column ROWKEY: INTEGER", + "message": "'10' can not be converted to the type of the key column: ROWKEY INTEGER KEY", "status": 400 } }, @@ -901,7 +901,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "Multiple bounds on ROWKEY", + "message": "Multiple bounds on key column", "status": 400 } }, @@ -914,7 +914,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "ROWKEY bound must currently be '='.", + "message": "Bound on 'ROWKEY' must currently be '='.", "status": 400 } }, @@ -927,7 +927,7 @@ ], "expectedError": { "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", - "message": "WHERE clause on unsupported field: COUNT", + "message": "WHERE clause on unsupported column: COUNT", "status": 400 } }, @@ -1015,6 +1015,29 @@ {"row":{"columns":[12345, 10.1, 12000, 13000]}} ]} ] + }, + { + "name": "should handled quoted key and value", + "statements": [ + "CREATE STREAM INPUT (`Key` STRING KEY, IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS `Value` FROM INPUT GROUP BY `Key`;", + "SELECT * FROM AGGREGATE WHERE `Key`='10';" + ], + "properties": { + "ksql.any.key.name.enabled": true + }, + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12365, "key": "10", "value": {}} + ], + "responses": [ + {"admin": {"@type": "currentStatus"}}, + {"admin": {"@type": "currentStatus"}}, + {"query": [ + {"header":{"schema":"`Key` STRING KEY, `ROWTIME` BIGINT, `Value` BIGINT"}}, + {"row":{"columns":["10", 12365, 1]}} + ]} + ] } ] } \ No newline at end of file diff --git a/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java b/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java index dfaf0cebe73b..48580eb185b1 100644 --- a/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java +++ b/ksql-metastore/src/test/java/io/confluent/ksql/metastore/model/StructuredDataSourceTest.java @@ -36,7 +36,7 @@ public class StructuredDataSourceTest { private static final LogicalSchema SOME_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); @@ -60,7 +60,7 @@ public void shouldThrowIfSchemaContainsRowTime() { // Given: final LogicalSchema schema = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(SchemaUtil.ROWTIME_NAME, SqlTypes.BIGINT) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); @@ -73,12 +73,11 @@ public void shouldThrowIfSchemaContainsRowTime() { } @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfSchemaContainsRowKey() { + public void shouldThrowOnDuplicateColumnNames() { // Given: final LogicalSchema schema = LogicalSchema.builder() - .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) - .valueColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("dup"), SqlTypes.INTEGER) + .valueColumn(ColumnName.of("dup"), SqlTypes.STRING) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); @@ -94,7 +93,7 @@ public void shouldThrowIfSchemaContainsWindowStart() { // Given: final LogicalSchema schema = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(SchemaUtil.WINDOWSTART_NAME, SqlTypes.STRING) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); @@ -111,7 +110,7 @@ public void shouldThrowIfSchemaContainsWindowEnd() { // Given: final LogicalSchema schema = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) .valueColumn(SchemaUtil.WINDOWEND_NAME, SqlTypes.STRING) .valueColumn(ColumnName.of("f0"), SqlTypes.BIGINT) .build(); @@ -123,6 +122,22 @@ public void shouldThrowIfSchemaContainsWindowEnd() { ); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfSchemaContainsValueColumnsWithSameNameAsKeyColumns() { + // Given: + final LogicalSchema schema = LogicalSchema.builder() + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) + .valueColumn(SchemaUtil.WINDOWEND_NAME, SqlTypes.STRING) + .valueColumn(ColumnName.of("k0"), SqlTypes.BIGINT) + .build(); + + // When: + new TestStructuredDataSource( + schema, + keyField + ); + } + /** * Test class to allow the abstract base class to be instantiated. */ diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java index b96e864a2878..95e5d21b1593 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java @@ -92,7 +92,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.math.BigDecimal; import java.util.List; import java.util.Objects; @@ -136,7 +135,7 @@ public class KsqlParserTest { .build(); private static final LogicalSchema ORDERS_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java index c07e8aef95a1..4d6eb9984ba8 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/SchemaParserTest.java @@ -72,7 +72,7 @@ public void shouldParseValidSchema() { } @Test - public void shouldParseValidSchemaWithKeyField() { + public void shouldParseValidSchemaWithRowKeyField() { // Given: final String schema = "ROWKEY STRING KEY, bar INT"; @@ -86,6 +86,21 @@ public void shouldParseValidSchemaWithKeyField() { )); } + @Test + public void shouldParseValidSchemaWithKeyField() { + // Given: + final String schema = "K STRING KEY, bar INT"; + + // When: + final TableElements elements = parser.parse(schema); + + // Then: + assertThat(elements, contains( + new TableElement(Namespace.KEY, ColumnName.of("K"), new Type(SqlTypes.STRING)), + new TableElement(Namespace.VALUE, BAR, new Type(SqlTypes.INTEGER)) + )); + } + @Test public void shouldParseQuotedSchema() { // Given: diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java index d2eef9bfe741..08a7a9556ef6 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/SqlFormatterTest.java @@ -67,7 +67,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.ValueFormat; import io.confluent.ksql.util.MetaStoreFixture; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import java.util.concurrent.TimeUnit; import org.junit.Before; @@ -109,7 +108,7 @@ public class SqlFormatterTest { .build(); private static final LogicalSchema ITEM_INFO_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K0"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("NAME"), SqlTypes.STRING) .valueColumn(ColumnName.of("CATEGORY"), categorySchema) @@ -122,7 +121,7 @@ public class SqlFormatterTest { .build(); private static final LogicalSchema ORDERS_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(ColumnName.of("K1"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORDERID"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) @@ -142,7 +141,7 @@ CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral("topic_test")) ); private static final TableElements ELEMENTS_WITH_KEY = TableElements.of( - new TableElement(Namespace.KEY, ColumnName.of("ROWKEY"), new Type(SqlTypes.STRING)), + new TableElement(Namespace.KEY, ColumnName.of("k3"), new Type(SqlTypes.STRING)), new TableElement(Namespace.VALUE, ColumnName.of("Foo"), new Type(SqlTypes.STRING)) ); @@ -170,7 +169,7 @@ public void setUp() { ValueFormat.of(FormatInfo.of(FormatFactory.JSON.name())) ); - final KsqlStream ksqlStreamOrders = new KsqlStream<>( + final KsqlStream ksqlStreamOrders = new KsqlStream<>( "sqlexpression", SourceName.of("ADDRESS"), ORDERS_SCHEMA, @@ -228,7 +227,7 @@ public void shouldFormatCreateStreamStatementWithExplicitKey() { final String sql = SqlFormatter.formatSql(createStream); // Then: - assertThat(sql, is("CREATE STREAM TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE STREAM TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', VALUE_FORMAT='JSON');")); } @@ -269,7 +268,7 @@ public void shouldFormatCreateTableStatementWithExplicitTimestamp() { final String sql = SqlFormatter.formatSql(createTable); // Then: - assertThat(sql, is("CREATE TABLE TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE TABLE TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', " + "TIMESTAMP='Foo', TIMESTAMP_FORMAT='%s', VALUE_FORMAT='JSON');")); } @@ -287,7 +286,7 @@ public void shouldFormatCreateTableStatementWithExplicitKey() { final String sql = SqlFormatter.formatSql(createTable); // Then: - assertThat(sql, is("CREATE TABLE TEST (ROWKEY STRING KEY, `Foo` STRING) " + assertThat(sql, is("CREATE TABLE TEST (`k3` STRING KEY, `Foo` STRING) " + "WITH (KAFKA_TOPIC='topic_test', KEY='ORDERID', VALUE_FORMAT='JSON');")); } diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java index 67a4b368111f..5c80a52b8e63 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementTest.java @@ -25,7 +25,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.NodeLocation; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.SchemaUtil; import java.util.Optional; import org.junit.Rule; import org.junit.Test; @@ -41,6 +40,7 @@ public class TableElementTest { @Rule public final ExpectedException expectedException = ExpectedException.none(); + @SuppressWarnings("UnstableApiUsage") @Test public void shouldImplementEquals() { new EqualsTester() @@ -55,7 +55,7 @@ public void shouldImplementEquals() { new TableElement(VALUE, NAME, new Type(SqlTypes.INTEGER)) ) .addEqualityGroup( - new TableElement(KEY, SchemaUtil.ROWKEY_NAME, new Type(SqlTypes.STRING)) + new TableElement(KEY, NAME, new Type(SqlTypes.STRING)) ) .testEquals(); } diff --git a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java index e16908b326e1..9248c9cb3fd5 100644 --- a/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java +++ b/ksql-parser/src/test/java/io/confluent/ksql/parser/tree/TableElementsTest.java @@ -223,6 +223,26 @@ public void shouldBuildLogicalSchemaWithImplicits() { )); } + @Test + public void shouldBuildLogicalSchemaWithImplicitsAndExplicitKey() { + // Given: + final TableElements tableElements = TableElements.of( + tableElement(VALUE, "v0", INT_TYPE), + tableElement(KEY, "k0", INT_TYPE) + ); + + // When: + final LogicalSchema schema = tableElements.toLogicalSchema(true); + + // Then: + assertThat(schema, is(LogicalSchema.builder() + .withRowTime() + .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) + .keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER) + .build() + )); + } + @Test public void shouldBuildLogicalSchemaWithOutImplicits() { // Given: diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java index b051fefb1717..341d54b222b5 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java @@ -308,7 +308,7 @@ public void shouldFailPullQueryWithNonKeyLookup() { String sql = "SELECT * from " + AGG_TABLE + " WHERE ROWTIME=12345;"; // Then: - shouldFailToExecuteQuery(sql, "WHERE clause on unsupported field: ROWTIME."); + shouldFailToExecuteQuery(sql, "WHERE clause on unsupported column: ROWTIME."); } @Test diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityFactoryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityFactoryTest.java index 6ba37b8dce02..a5e316bcf2c5 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityFactoryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/TableRowsEntityFactoryTest.java @@ -41,17 +41,20 @@ public class TableRowsEntityFactoryTest { - private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.STRING); + private static final ColumnName K0 = ColumnName.of("k0"); + + private static final KeyBuilder KEY_BUILDER = StructKeyUtil + .keyBuilder(K0, SqlTypes.STRING); private static final LogicalSchema SIMPLE_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.BOOLEAN) .build(); private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .keyColumn(ColumnName.of("k1"), SqlTypes.BOOLEAN) .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("v1"), SqlTypes.BOOLEAN) @@ -59,7 +62,7 @@ public class TableRowsEntityFactoryTest { private static final LogicalSchema SCHEMA_NULL = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v1"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("v2"), SqlTypes.DOUBLE) @@ -74,7 +77,7 @@ public void shouldAddNonWindowedRowToValues() { final List input = ImmutableList.of( Row.of( SIMPLE_SCHEMA, - STRING_KEY_BUILDER.build("x"), + KEY_BUILDER.build("x"), genericRow(false), ROWTIME ) @@ -98,13 +101,13 @@ public void shouldAddWindowedRowToValues() { final List input = ImmutableList.of( WindowedRow.of( SIMPLE_SCHEMA, - new Windowed<>(STRING_KEY_BUILDER.build("x"), window0), + new Windowed<>(KEY_BUILDER.build("x"), window0), genericRow(true), ROWTIME ), WindowedRow.of( SIMPLE_SCHEMA, - new Windowed<>(STRING_KEY_BUILDER.build("y"), window1), + new Windowed<>(KEY_BUILDER.build("y"), window1), genericRow(false), ROWTIME ) @@ -127,7 +130,7 @@ public void shouldSupportNullColumns() { final GenericRow row = genericRow(null, null, null, null); final Builder builder = ImmutableList.builder(); - builder.add(Row.of(SCHEMA_NULL, STRING_KEY_BUILDER.build("k"), row, ROWTIME)); + builder.add(Row.of(SCHEMA_NULL, KEY_BUILDER.build("k"), row, ROWTIME)); // When: final List> output = TableRowsEntityFactory.createRows(builder.build()); @@ -144,7 +147,7 @@ public void shouldJustDuplicateRowTimeInValueIfNotWindowed() { // Then: assertThat(result, is(LogicalSchema.builder() - .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .keyColumn(ColumnName.of("k1"), SqlTypes.BOOLEAN) .valueColumn(SchemaUtil.ROWTIME_NAME, SqlTypes.BIGINT) .valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER) @@ -160,7 +163,7 @@ public void shouldAddHoppingWindowFieldsToSchema() { // Then: assertThat(result, is(LogicalSchema.builder() - .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) + .keyColumn(K0, SqlTypes.STRING) .keyColumn(ColumnName.of("k1"), SqlTypes.BOOLEAN) .keyColumn(ColumnName.of("WINDOWSTART"), SqlTypes.BIGINT) .keyColumn(ColumnName.of("WINDOWEND"), SqlTypes.BIGINT) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/GroupByParamsFactoryTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/GroupByParamsFactoryTest.java index 641d136de504..39596366b1a2 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/GroupByParamsFactoryTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/GroupByParamsFactoryTest.java @@ -38,6 +38,7 @@ import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.util.SchemaUtil; import java.util.Collections; import java.util.List; import java.util.function.Function; @@ -54,8 +55,10 @@ @RunWith(MockitoJUnitRunner.class) public class GroupByParamsFactoryTest { - private static final KeyBuilder INT_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.INTEGER); - private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.STRING); + private static final KeyBuilder INT_KEY_BUILDER = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.INTEGER); + private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING); private static final LogicalSchema SOURCE_SCHEMA = LogicalSchema.builder() .valueColumn(ColumnName.of("v0"), SqlTypes.DOUBLE) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java index 8398b8fe2247..82154fa4dff2 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/JoinParamsFactoryTest.java @@ -9,7 +9,6 @@ import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; -import io.confluent.ksql.util.SchemaUtil; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -17,13 +16,13 @@ public class JoinParamsFactoryTest { private static final LogicalSchema LEFT_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("LK"), SqlTypes.STRING) .valueColumn(ColumnName.of("BLUE"), SqlTypes.STRING) .valueColumn(ColumnName.of("GREEN"), SqlTypes.INTEGER) .build(); private static final LogicalSchema RIGHT_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("RK"), SqlTypes.STRING) .valueColumn(ColumnName.of("RED"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.DOUBLE) .build(); @@ -41,7 +40,7 @@ public void shouldBuildCorrectSchema() { // Then: assertThat(joinParams.getSchema(), is(LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("LK"), SqlTypes.STRING) .valueColumn(ColumnName.of("BLUE"), SqlTypes.STRING) .valueColumn(ColumnName.of("GREEN"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("RED"), SqlTypes.BIGINT) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java index b8d6f9a8bd9d..90c1fbcfda5a 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java @@ -58,7 +58,6 @@ import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.WindowInfo; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.SchemaUtil; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -98,20 +97,22 @@ public class SourceBuilderTest { + private static final ColumnName K0 = ColumnName.of("k0"); + private static final LogicalSchema SOURCE_SCHEMA = LogicalSchema.builder() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.BIGINT) + .keyColumn(K0, SqlTypes.BIGINT) .valueColumn(ColumnName.of("field1"), SqlTypes.STRING) .valueColumn(ColumnName.of("field2"), SqlTypes.BIGINT) .build(); private static final Schema KEY_SCHEMA = SchemaBuilder.struct() - .field(SchemaUtil.ROWKEY_NAME.text(), Schema.OPTIONAL_FLOAT64_SCHEMA) + .field(K0.text(), Schema.OPTIONAL_FLOAT64_SCHEMA) .build(); private static final double A_KEY = 10.11; private static final Struct KEY = new Struct(KEY_SCHEMA) - .put(SchemaUtil.ROWKEY_NAME.text(), A_KEY); + .put(K0.text(), A_KEY); private static final LogicalSchema SCHEMA = SOURCE_SCHEMA .withMetaAndKeyColsInValue(false); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java index 9b0b32b46668..0dcff5b05fe4 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StepSchemaResolverTest.java @@ -85,7 +85,7 @@ public class StepSchemaResolverTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("APPLE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BANANA"), SqlTypes.STRING) @@ -136,7 +136,7 @@ public void shouldResolveSchemaForStreamAggregate() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .build()) @@ -146,13 +146,13 @@ public void shouldResolveSchemaForStreamAggregate() { @Test public void shouldResolveSchemaForStreamWindowedAggregate() { // Given: - givenAggregateFunction("SUM", SqlTypes.BIGINT); + givenAggregateFunction("COUNT", SqlTypes.BIGINT); final StreamWindowedAggregate step = new StreamWindowedAggregate( PROPERTIES, groupedStreamSource, formats, ImmutableList.of(ColumnName.of("ORANGE")), - ImmutableList.of(functionCall("SUM", "APPLE")), + ImmutableList.of(functionCall("COUNT", "APPLE")), new TumblingWindowExpression(10, TimeUnit.SECONDS) ); @@ -163,7 +163,7 @@ public void shouldResolveSchemaForStreamWindowedAggregate() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .valueColumn(SchemaUtil.WINDOWSTART_NAME, SchemaUtil.WINDOWBOUND_TYPE) @@ -191,7 +191,7 @@ public void shouldResolveSchemaForStreamSelect() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("JUICE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("PLANTAIN"), SqlTypes.STRING) .valueColumn(ColumnName.of("CITRUS"), SqlTypes.INTEGER) @@ -216,7 +216,7 @@ public void shouldResolveSchemaForStreamFlatMap() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("APPLE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BANANA"), SqlTypes.STRING) @@ -356,7 +356,7 @@ public void shouldResolveSchemaForTableAggregate() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("ORANGE"), SqlTypes.INTEGER) .valueColumn(ColumnName.aggregateColumn(0), SqlTypes.BIGINT) .build()) @@ -403,7 +403,7 @@ public void shouldResolveSchemaForTableSelect() { assertThat(result, is( LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("JUICE"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("PLANTAIN"), SqlTypes.STRING) .valueColumn(ColumnName.of("CITRUS"), SqlTypes.INTEGER) diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java index 91946e8ab6d7..593d6916a4e3 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamGroupByBuilderTest.java @@ -58,11 +58,11 @@ public class StreamGroupByBuilderTest { - private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil.keyBuilder(SqlTypes.STRING); - + private static final KeyBuilder STRING_KEY_BUILDER = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING); private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.INTEGER) .valueColumn(ColumnName.of("PAC"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("MAN"), SqlTypes.STRING) .build() diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java index b80151a8370a..5b1c24e6c8a1 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSelectKeyBuilderTest.java @@ -64,7 +64,7 @@ public class StreamSelectKeyBuilderTest { private static final LogicalSchema SOURCE_SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("BIG"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BOI"), SqlTypes.BIGINT) .build() @@ -79,17 +79,16 @@ public class StreamSelectKeyBuilderTest { .valueColumn(ColumnName.of("BIG"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("BOI"), SqlTypes.BIGINT) .valueColumn(ColumnName.of(SchemaUtil.ROWTIME_NAME.text()), SqlTypes.BIGINT) - // Note: Type of ROWKEY is old key's type: - .valueColumn(ColumnName.of(SchemaUtil.ROWKEY_NAME.text()), SqlTypes.STRING) + .valueColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .build(); private static final KeyBuilder RESULT_KEY_BUILDER = StructKeyUtil.keyBuilder(RESULT_SCHEMA); - private static final long A_BOI = 5000; private static final long A_BIG = 3000; - private static final Struct SOURCE_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING) + private static final Struct SOURCE_KEY = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) .build("dre"); @Mock diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java index 8c1bc47a8580..46f6ad0202bd 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableGroupByBuilderTest.java @@ -59,7 +59,7 @@ public class TableGroupByBuilderTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.DOUBLE) .valueColumn(ColumnName.of("PAC"), SqlTypes.BIGINT) .valueColumn(ColumnName.of("MAN"), SqlTypes.STRING) .build() @@ -94,7 +94,8 @@ public class TableGroupByBuilderTest { SerdeOption.none() ); - private static final Struct KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING).build("key"); + private static final Struct KEY = StructKeyUtil + .keyBuilder(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING).build("key"); @Mock private KsqlQueryBuilder queryBuilder; diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java index 5bba6ac331a1..ae3e71e4a9c4 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/KsqlMaterializationTest.java @@ -41,7 +41,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.SchemaUtil; import java.time.Instant; import java.util.List; import java.util.Optional; @@ -61,12 +60,13 @@ public class KsqlMaterializationTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v1"), SqlTypes.STRING) .build(); - private static final Struct A_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING).build("k"); + private static final Struct A_KEY = StructKeyUtil + .keyBuilder(ColumnName.of("k0"), SqlTypes.STRING).build("k"); private static final long A_ROWTIME = 12335L; private static final Range WINDOW_START_BOUNDS = Range.closed( Instant.now(), diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedSessionTableTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedSessionTableTest.java index 7b6692dd1bbb..dca2ad23cc33 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedSessionTableTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedSessionTableTest.java @@ -59,11 +59,12 @@ public class KsMaterializedSessionTableTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("ROWKEY"), SqlTypes.STRING) + .keyColumn(ColumnName.of("k0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .build(); - private static final Struct A_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING).build("x"); + private static final Struct A_KEY = StructKeyUtil + .keyBuilder(ColumnName.of("k0"), SqlTypes.STRING).build("x"); private static final GenericRow A_VALUE = GenericRow.genericRow("c0l"); private static final Instant LOWER_INSTANT = Instant.now(); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedTableTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedTableTest.java index 6da1d419a936..96133d768b81 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedTableTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedTableTest.java @@ -52,11 +52,12 @@ public class KsMaterializedTableTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(ColumnName.of("ROWKEY"), SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .build(); - private static final Struct A_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING).build("x"); + private static final Struct A_KEY = StructKeyUtil + .keyBuilder(ColumnName.of("K0"), SqlTypes.STRING).build("x"); @Rule public final ExpectedException expectedException = ExpectedException.none(); diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedWindowTableTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedWindowTableTest.java index 4f49a2e850f2..2f4ea2a685ed 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedWindowTableTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsMaterializedWindowTableTest.java @@ -35,7 +35,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.SchemaUtil; import java.time.Duration; import java.time.Instant; import java.util.List; @@ -64,11 +63,12 @@ public class KsMaterializedWindowTableTest { private static final LogicalSchema SCHEMA = LogicalSchema.builder() .withRowTime() - .keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING) + .keyColumn(ColumnName.of("K0"), SqlTypes.STRING) .valueColumn(ColumnName.of("v0"), SqlTypes.STRING) .build(); - private static final Struct A_KEY = StructKeyUtil.keyBuilder(SqlTypes.STRING).build("x"); + private static final Struct A_KEY = StructKeyUtil + .keyBuilder(ColumnName.of("K0"), SqlTypes.STRING).build("x"); private static final Range WINDOW_START_BOUNDS = Range.closed( Instant.now(), From e3cae9c406742390c00e253dc0921d5f9f076ab3 Mon Sep 17 00:00:00 2001 From: Andy Coates Date: Wed, 4 Mar 2020 13:42:02 +0000 Subject: [PATCH 3/3] chore: commit historical plans --- .../6.0.0_1583319039183/plan.json | 147 +++++++ .../6.0.0_1583319039183/spec.json | 22 + .../6.0.0_1583319039183/topology | 13 + .../6.0.0_1583329131267/plan.json | 190 +++++++++ .../6.0.0_1583329131267/spec.json | 69 ++++ .../6.0.0_1583329131267/topology | 25 ++ .../6.0.0_1583319041765/plan.json | 190 +++++++++ .../6.0.0_1583319041765/spec.json | 52 +++ .../6.0.0_1583319041765/topology | 25 ++ .../6.0.0_1583329131307/plan.json | 190 +++++++++ .../6.0.0_1583329131307/spec.json | 69 ++++ .../6.0.0_1583329131307/topology | 25 ++ .../6.0.0_1583319045585/plan.json | 351 ++++++++++++++++ .../6.0.0_1583319045585/spec.json | 63 +++ .../6.0.0_1583319045585/topology | 42 ++ .../6.0.0_1583319046811/plan.json | 253 ++++++++++++ .../6.0.0_1583319046811/spec.json | 62 +++ .../6.0.0_1583319046811/topology | 39 ++ .../6.0.0_1583319046881/plan.json | 253 ++++++++++++ .../6.0.0_1583319046881/spec.json | 115 ++++++ .../6.0.0_1583319046881/topology | 39 ++ .../6.0.0_1583319046422/plan.json | 250 ++++++++++++ .../6.0.0_1583319046422/spec.json | 38 ++ .../6.0.0_1583319046422/topology | 69 ++++ .../6.0.0_1583319046460/plan.json | 250 ++++++++++++ .../6.0.0_1583319046460/spec.json | 38 ++ .../6.0.0_1583319046460/topology | 69 ++++ .../6.0.0_1583319046499/plan.json | 250 ++++++++++++ .../6.0.0_1583319046499/spec.json | 38 ++ .../6.0.0_1583319046499/topology | 69 ++++ .../6.0.0_1583319046539/plan.json | 250 ++++++++++++ .../6.0.0_1583319046539/spec.json | 38 ++ .../6.0.0_1583319046539/topology | 69 ++++ .../6.0.0_1583319046589/plan.json | 250 ++++++++++++ .../6.0.0_1583319046589/spec.json | 38 ++ .../6.0.0_1583319046589/topology | 69 ++++ .../6.0.0_1583319046631/plan.json | 250 ++++++++++++ .../6.0.0_1583319046631/spec.json | 38 ++ .../6.0.0_1583319046631/topology | 69 ++++ .../6.0.0_1583319046311/plan.json | 250 ++++++++++++ .../6.0.0_1583319046311/spec.json | 38 ++ .../6.0.0_1583319046311/topology | 69 ++++ .../6.0.0_1583319046347/plan.json | 250 ++++++++++++ .../6.0.0_1583319046347/spec.json | 38 ++ .../6.0.0_1583319046347/topology | 69 ++++ .../6.0.0_1583319046384/plan.json | 250 ++++++++++++ .../6.0.0_1583319046384/spec.json | 38 ++ .../6.0.0_1583319046384/topology | 69 ++++ .../6.0.0_1583319046673/plan.json | 250 ++++++++++++ .../6.0.0_1583319046673/spec.json | 38 ++ .../6.0.0_1583319046673/topology | 69 ++++ .../6.0.0_1583319046719/plan.json | 250 ++++++++++++ .../6.0.0_1583319046719/spec.json | 38 ++ .../6.0.0_1583319046719/topology | 69 ++++ .../6.0.0_1583319046763/plan.json | 250 ++++++++++++ .../6.0.0_1583319046763/spec.json | 38 ++ .../6.0.0_1583319046763/topology | 69 ++++ .../6.0.0_1583319046266/plan.json | 230 +++++++++++ .../6.0.0_1583319046266/spec.json | 52 +++ .../6.0.0_1583319046266/topology | 45 +++ .../6.0.0_1583319046239/plan.json | 236 +++++++++++ .../6.0.0_1583319046239/spec.json | 125 ++++++ .../6.0.0_1583319046239/topology | 39 ++ .../6.0.0_1583319043960/plan.json | 236 +++++++++++ .../6.0.0_1583319043960/spec.json | 107 +++++ .../6.0.0_1583319043960/topology | 39 ++ .../6.0.0_1583319044009/plan.json | 236 +++++++++++ .../6.0.0_1583319044009/spec.json | 107 +++++ .../6.0.0_1583319044009/topology | 39 ++ .../6.0.0_1583319044058/plan.json | 236 +++++++++++ .../6.0.0_1583319044058/spec.json | 107 +++++ .../6.0.0_1583319044058/topology | 39 ++ .../6.0.0_1583319044370/plan.json | 236 +++++++++++ .../6.0.0_1583319044370/spec.json | 217 ++++++++++ .../6.0.0_1583319044370/topology | 39 ++ .../6.0.0_1583319044415/plan.json | 236 +++++++++++ .../6.0.0_1583319044415/spec.json | 217 ++++++++++ .../6.0.0_1583319044415/topology | 39 ++ .../6.0.0_1583319044454/plan.json | 236 +++++++++++ .../6.0.0_1583319044454/spec.json | 217 ++++++++++ .../6.0.0_1583319044454/topology | 39 ++ .../6.0.0_1583319044111/plan.json | 236 +++++++++++ .../6.0.0_1583319044111/spec.json | 110 +++++ .../6.0.0_1583319044111/topology | 39 ++ .../6.0.0_1583319044151/plan.json | 236 +++++++++++ .../6.0.0_1583319044151/spec.json | 110 +++++ .../6.0.0_1583319044151/topology | 39 ++ .../6.0.0_1583319044195/plan.json | 236 +++++++++++ .../6.0.0_1583319044195/spec.json | 110 +++++ .../6.0.0_1583319044195/topology | 39 ++ .../6.0.0_1583319044242/plan.json | 236 +++++++++++ .../6.0.0_1583319044242/spec.json | 110 +++++ .../6.0.0_1583319044242/topology | 39 ++ .../6.0.0_1583319044289/plan.json | 236 +++++++++++ .../6.0.0_1583319044289/spec.json | 110 +++++ .../6.0.0_1583319044289/topology | 39 ++ .../6.0.0_1583319044330/plan.json | 236 +++++++++++ .../6.0.0_1583319044330/spec.json | 110 +++++ .../6.0.0_1583319044330/topology | 39 ++ .../6.0.0_1583319044492/plan.json | 236 +++++++++++ .../6.0.0_1583319044492/spec.json | 99 +++++ .../6.0.0_1583319044492/topology | 39 ++ .../6.0.0_1583319044534/plan.json | 236 +++++++++++ .../6.0.0_1583319044534/spec.json | 99 +++++ .../6.0.0_1583319044534/topology | 39 ++ .../6.0.0_1583319044577/plan.json | 236 +++++++++++ .../6.0.0_1583319044577/spec.json | 99 +++++ .../6.0.0_1583319044577/topology | 39 ++ .../6.0.0_1583319044623/plan.json | 236 +++++++++++ .../6.0.0_1583319044623/spec.json | 140 +++++++ .../6.0.0_1583319044623/topology | 39 ++ .../6.0.0_1583319044669/plan.json | 236 +++++++++++ .../6.0.0_1583319044669/spec.json | 140 +++++++ .../6.0.0_1583319044669/topology | 39 ++ .../6.0.0_1583319044712/plan.json | 236 +++++++++++ .../6.0.0_1583319044712/spec.json | 140 +++++++ .../6.0.0_1583319044712/topology | 39 ++ .../6.0.0_1583319046181/plan.json | 243 +++++++++++ .../6.0.0_1583319046181/spec.json | 35 ++ .../6.0.0_1583319046181/topology | 54 +++ .../6.0.0_1583319046030/plan.json | 243 +++++++++++ .../6.0.0_1583319046030/spec.json | 43 ++ .../6.0.0_1583319046030/topology | 54 +++ .../6.0.0_1583319046064/plan.json | 243 +++++++++++ .../6.0.0_1583319046064/spec.json | 34 ++ .../6.0.0_1583319046064/topology | 54 +++ .../6.0.0_1583319046120/plan.json | 243 +++++++++++ .../6.0.0_1583319046120/spec.json | 35 ++ .../6.0.0_1583319046120/topology | 54 +++ .../6.0.0_1583319046150/plan.json | 243 +++++++++++ .../6.0.0_1583319046150/spec.json | 35 ++ .../6.0.0_1583319046150/topology | 54 +++ .../6.0.0_1583319046211/plan.json | 243 +++++++++++ .../6.0.0_1583319046211/spec.json | 30 ++ .../6.0.0_1583319046211/topology | 54 +++ .../6.0.0_1583319046001/plan.json | 250 ++++++++++++ .../6.0.0_1583319046001/spec.json | 34 ++ .../6.0.0_1583319046001/topology | 69 ++++ .../6.0.0_1583319046093/plan.json | 243 +++++++++++ .../6.0.0_1583319046093/spec.json | 41 ++ .../6.0.0_1583319046093/topology | 54 +++ .../6.0.0_1583319043519/plan.json | 236 +++++++++++ .../6.0.0_1583319043519/spec.json | 267 +++++++++++++ .../6.0.0_1583319043519/topology | 39 ++ .../6.0.0_1583319043567/plan.json | 236 +++++++++++ .../6.0.0_1583319043567/spec.json | 267 +++++++++++++ .../6.0.0_1583319043567/topology | 39 ++ .../6.0.0_1583319043603/plan.json | 236 +++++++++++ .../6.0.0_1583319043603/spec.json | 267 +++++++++++++ .../6.0.0_1583319043603/topology | 39 ++ .../6.0.0_1583319043796/plan.json | 250 ++++++++++++ .../6.0.0_1583319043796/spec.json | 378 ++++++++++++++++++ .../6.0.0_1583319043796/topology | 69 ++++ .../6.0.0_1583319043850/plan.json | 250 ++++++++++++ .../6.0.0_1583319043850/spec.json | 378 ++++++++++++++++++ .../6.0.0_1583319043850/topology | 69 ++++ .../6.0.0_1583319043903/plan.json | 250 ++++++++++++ .../6.0.0_1583319043903/spec.json | 378 ++++++++++++++++++ .../6.0.0_1583319043903/topology | 69 ++++ .../6.0.0_1583319043641/plan.json | 250 ++++++++++++ .../6.0.0_1583319043641/spec.json | 162 ++++++++ .../6.0.0_1583319043641/topology | 69 ++++ .../6.0.0_1583319043696/plan.json | 250 ++++++++++++ .../6.0.0_1583319043696/spec.json | 162 ++++++++ .../6.0.0_1583319043696/topology | 69 ++++ .../6.0.0_1583319043742/plan.json | 250 ++++++++++++ .../6.0.0_1583319043742/spec.json | 169 ++++++++ .../6.0.0_1583319043742/topology | 69 ++++ .../6.0.0_1583319044758/plan.json | 236 +++++++++++ .../6.0.0_1583319044758/spec.json | 165 ++++++++ .../6.0.0_1583319044758/topology | 39 ++ .../6.0.0_1583319044802/plan.json | 236 +++++++++++ .../6.0.0_1583319044802/spec.json | 165 ++++++++ .../6.0.0_1583319044802/topology | 39 ++ .../6.0.0_1583319044848/plan.json | 236 +++++++++++ .../6.0.0_1583319044848/spec.json | 165 ++++++++ .../6.0.0_1583319044848/topology | 39 ++ .../6.0.0_1583319045735/plan.json | 236 +++++++++++ .../6.0.0_1583319045735/spec.json | 41 ++ .../6.0.0_1583319045735/topology | 39 ++ .../6.0.0_1583319045768/plan.json | 236 +++++++++++ .../6.0.0_1583319045768/spec.json | 38 ++ .../6.0.0_1583319045768/topology | 39 ++ .../6.0.0_1583319045708/plan.json | 236 +++++++++++ .../6.0.0_1583319045708/spec.json | 73 ++++ .../6.0.0_1583319045708/topology | 39 ++ .../6.0.0_1583319045441/plan.json | 223 +++++++++++ .../6.0.0_1583319045441/spec.json | 101 +++++ .../6.0.0_1583319045441/topology | 30 ++ .../6.0.0_1583319045487/plan.json | 223 +++++++++++ .../6.0.0_1583319045487/spec.json | 101 +++++ .../6.0.0_1583319045487/topology | 30 ++ .../6.0.0_1583319045537/plan.json | 223 +++++++++++ .../6.0.0_1583319045537/spec.json | 101 +++++ .../6.0.0_1583319045537/topology | 30 ++ .../6.0.0_1583319045300/plan.json | 223 +++++++++++ .../6.0.0_1583319045300/spec.json | 112 ++++++ .../6.0.0_1583319045300/topology | 30 ++ .../6.0.0_1583319045348/plan.json | 223 +++++++++++ .../6.0.0_1583319045348/spec.json | 112 ++++++ .../6.0.0_1583319045348/topology | 30 ++ .../6.0.0_1583319045397/plan.json | 223 +++++++++++ .../6.0.0_1583319045397/spec.json | 112 ++++++ .../6.0.0_1583319045397/topology | 30 ++ .../6.0.0_1583319045835/plan.json | 223 +++++++++++ .../6.0.0_1583319045835/spec.json | 45 +++ .../6.0.0_1583319045835/topology | 30 ++ .../6.0.0_1583319045871/plan.json | 223 +++++++++++ .../6.0.0_1583319045871/spec.json | 42 ++ .../6.0.0_1583319045871/topology | 30 ++ .../6.0.0_1583319045798/plan.json | 223 +++++++++++ .../6.0.0_1583319045798/spec.json | 83 ++++ .../6.0.0_1583319045798/topology | 30 ++ .../6.0.0_1583319045047/plan.json | 212 ++++++++++ .../6.0.0_1583319045047/spec.json | 119 ++++++ .../6.0.0_1583319045047/topology | 42 ++ .../6.0.0_1583319045090/plan.json | 212 ++++++++++ .../6.0.0_1583319045090/spec.json | 119 ++++++ .../6.0.0_1583319045090/topology | 42 ++ .../6.0.0_1583319045132/plan.json | 212 ++++++++++ .../6.0.0_1583319045132/spec.json | 119 ++++++ .../6.0.0_1583319045132/topology | 42 ++ .../6.0.0_1583319045667/plan.json | 219 ++++++++++ .../6.0.0_1583319045667/spec.json | 117 ++++++ .../6.0.0_1583319045667/topology | 51 +++ .../6.0.0_1583319044900/plan.json | 212 ++++++++++ .../6.0.0_1583319044900/spec.json | 144 +++++++ .../6.0.0_1583319044900/topology | 42 ++ .../6.0.0_1583319044961/plan.json | 212 ++++++++++ .../6.0.0_1583319044961/spec.json | 144 +++++++ .../6.0.0_1583319044961/topology | 42 ++ .../6.0.0_1583319045007/plan.json | 212 ++++++++++ .../6.0.0_1583319045007/spec.json | 144 +++++++ .../6.0.0_1583319045007/topology | 42 ++ .../6.0.0_1583319045174/plan.json | 212 ++++++++++ .../6.0.0_1583319045174/spec.json | 144 +++++++ .../6.0.0_1583319045174/topology | 42 ++ .../6.0.0_1583319045215/plan.json | 212 ++++++++++ .../6.0.0_1583319045215/spec.json | 144 +++++++ .../6.0.0_1583319045215/topology | 42 ++ .../6.0.0_1583319045259/plan.json | 212 ++++++++++ .../6.0.0_1583319045259/spec.json | 144 +++++++ .../6.0.0_1583319045259/topology | 42 ++ .../6.0.0_1583319045946/plan.json | 212 ++++++++++ .../6.0.0_1583319045946/spec.json | 49 +++ .../6.0.0_1583319045946/topology | 42 ++ .../6.0.0_1583319045976/plan.json | 212 ++++++++++ .../6.0.0_1583319045976/spec.json | 46 +++ .../6.0.0_1583319045976/topology | 42 ++ .../6.0.0_1583319045918/plan.json | 212 ++++++++++ .../6.0.0_1583319045918/spec.json | 93 +++++ .../6.0.0_1583319045918/topology | 42 ++ .../6.0.0_1583166643536/plan.json | 145 +++++++ .../6.0.0_1583166643536/spec.json | 23 ++ .../6.0.0_1583166643536/topology | 13 + .../6.0.0_1583166643478/plan.json | 145 +++++++ .../6.0.0_1583166643478/spec.json | 23 ++ .../6.0.0_1583166643478/topology | 13 + .../6.0.0_1583329143853/plan.json | 147 +++++++ .../6.0.0_1583329143853/spec.json | 49 +++ .../6.0.0_1583329143853/topology | 13 + .../6.0.0_1583319053891/plan.json | 147 +++++++ .../6.0.0_1583319053891/spec.json | 22 + .../6.0.0_1583319053891/topology | 13 + .../6.0.0_1583329143863/plan.json | 147 +++++++ .../6.0.0_1583329143863/spec.json | 52 +++ .../6.0.0_1583329143863/topology | 13 + .../6.0.0_1583319054694/plan.json | 147 +++++++ .../6.0.0_1583319054694/spec.json | 26 ++ .../6.0.0_1583319054694/topology | 13 + .../6.0.0_1583319054687/plan.json | 147 +++++++ .../6.0.0_1583319054687/spec.json | 22 + .../6.0.0_1583319054687/topology | 13 + .../6.0.0_1583319054671/plan.json | 147 +++++++ .../6.0.0_1583319054671/spec.json | 22 + .../6.0.0_1583319054671/topology | 13 + .../6.0.0_1583319054680/plan.json | 147 +++++++ .../6.0.0_1583319054680/spec.json | 22 + .../6.0.0_1583319054680/topology | 13 + .../6.0.0_1583319056601/plan.json | 154 +++++++ .../6.0.0_1583319056601/spec.json | 22 + .../6.0.0_1583319056601/topology | 16 + .../6.0.0_1583319057678/plan.json | 154 +++++++ .../6.0.0_1583319057678/spec.json | 46 +++ .../6.0.0_1583319057678/topology | 16 + .../6.0.0_1583319057687/plan.json | 154 +++++++ .../6.0.0_1583319057687/spec.json | 46 +++ .../6.0.0_1583319057687/topology | 16 + 288 files changed, 34944 insertions(+) create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/topology create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/plan.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/spec.json create mode 100644 ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/topology diff --git a/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/plan.json new file mode 100644 index 000000000000..c505c69d4679 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, C1 BIGINT) WITH (KAFKA_TOPIC='input', SCHEMA_ID=1, VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `C1` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `C1` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `C1` BIGINT" + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/spec.json new file mode 100644 index 000000000000..6b69a56f6839 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319039183, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : 42, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 42, + "value" : { + "C1" : 4 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/topology b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_value_elements_OK_-_custom_key_name/6.0.0_1583319039183/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/plan.json new file mode 100644 index 000000000000..8c2a924366a0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/plan.json @@ -0,0 +1,190 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, IGNORED INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `IGNORED` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT COUNT(1) COUNT\nFROM INPUT INPUT\nGROUP BY INPUT.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `COUNT` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` INTEGER" + }, + "selectExpressions" : [ "ID AS KSQL_INTERNAL_COL_0", "1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent335545957785424598", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/spec.json new file mode 100644 index 000000000000..14169aac4538 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/spec.json @@ -0,0 +1,69 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583329131267, + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 11, + "value" : { }, + "timestamp" : 12345 + }, { + "topic" : "test_topic", + "key" : 10, + "value" : { }, + "timestamp" : 12365 + }, { + "topic" : "test_topic", + "key" : 11, + "value" : { }, + "timestamp" : 12375 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 11, + "value" : { + "KSQL_INTERNAL_COL_0" : 11, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 12345 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 10, + "value" : { + "KSQL_INTERNAL_COL_0" : 10, + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 12365 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : 11, + "value" : { + "KSQL_INTERNAL_COL_0" : 11, + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 12375 + }, { + "topic" : "OUTPUT", + "key" : 11, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "COUNT" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : 11, + "value" : { + "COUNT" : 2 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/topology b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column/6.0.0_1583329131267/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/plan.json new file mode 100644 index 000000000000..2749e6f4d631 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/plan.json @@ -0,0 +1,190 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, IGNORED STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `IGNORED` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT COUNT(*) KSQL_COL_0\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `KSQL_COL_0` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "ROWTIME AS KSQL_INTERNAL_COL_0", "ID AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0", "KSQL_INTERNAL_COL_1" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_0)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/spec.json new file mode 100644 index 000000000000..52f2f4262ed7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319041765, + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : "-" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "-" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "-" + }, { + "topic" : "test_topic", + "key" : 2, + "value" : "-" + }, { + "topic" : "test_topic", + "key" : 1, + "value" : "-" + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "1" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : "2" + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : "3" + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/topology b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_only_key_column_(stream-_table)/6.0.0_1583319041765/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/plan.json new file mode 100644 index 000000000000..62eca6532144 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/plan.json @@ -0,0 +1,190 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (`Key` STRING KEY, IGNORED INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`Key` STRING KEY, `IGNORED` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT COUNT(1) `Value`\nFROM INPUT INPUT\nGROUP BY INPUT.`Key`\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`Key` STRING KEY, `Value` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`Key` STRING KEY, `IGNORED` INTEGER" + }, + "selectExpressions" : [ "`Key` AS KSQL_INTERNAL_COL_0", "1 AS KSQL_INTERNAL_COL_1" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "nonAggregateColumns" : [ "KSQL_INTERNAL_COL_0" ], + "aggregationFunctions" : [ "COUNT(KSQL_INTERNAL_COL_1)" ] + }, + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS `Value`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent335545957785424598", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/spec.json new file mode 100644 index 000000000000..901902f16f07 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/spec.json @@ -0,0 +1,69 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583329131307, + "schemas" : { + "CTAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.GroupBy" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.Aggregate.Aggregate.Materialize" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "11", + "value" : { }, + "timestamp" : 12345 + }, { + "topic" : "test_topic", + "key" : "10", + "value" : { }, + "timestamp" : 12365 + }, { + "topic" : "test_topic", + "key" : "11", + "value" : { }, + "timestamp" : 12375 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "11", + "value" : { + "KSQL_INTERNAL_COL_0" : "11", + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 12345 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "10", + "value" : { + "KSQL_INTERNAL_COL_0" : "10", + "KSQL_AGG_VARIABLE_0" : 1 + }, + "timestamp" : 12365 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Aggregate-Aggregate-Materialize-changelog", + "key" : "11", + "value" : { + "KSQL_INTERNAL_COL_0" : "11", + "KSQL_AGG_VARIABLE_0" : 2 + }, + "timestamp" : 12375 + }, { + "topic" : "OUTPUT", + "key" : "11", + "value" : { + "Value" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "10", + "value" : { + "Value" : 1 + } + }, { + "topic" : "OUTPUT", + "key" : "11", + "value" : { + "Value" : 2 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/topology b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/topology new file mode 100644 index 000000000000..e000b160c4db --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/group-by_-_should_handled_quoted_key_and_value/6.0.0_1583329131307/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/plan.json new file mode 100644 index 000000000000..eec168843031 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/plan.json @@ -0,0 +1,351 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE_2 (ID BIGINT KEY, F3 STRING) WITH (KAFKA_TOPIC='right_topic_2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE_2", + "schema" : "`ID` BIGINT KEY, `F3` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic_2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN WITH (PARTITIONS=4) AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_0" + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN_2 AS SELECT\n TT.NAME NAME,\n TT.F1 F1,\n T.F3 F3\nFROM INNER_JOIN TT\nINNER JOIN TEST_TABLE_2 T ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN_2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `F1` STRING, `F3` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN_2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INNER_JOIN", "TEST_TABLE_2" ], + "sink" : "INNER_JOIN_2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN_2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "VALUE AS TT_VALUE", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic_2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F3` STRING" + }, + "selectExpressions" : [ "F3 AS T_F3", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + } + }, + "selectExpressions" : [ "TT_NAME AS NAME", "TT_F1 AS F1", "T_F3 AS F3" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN_2", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_2_1" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/spec.json new file mode 100644 index 000000000000..1519fd50bbac --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/spec.json @@ -0,0 +1,63 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045585, + "schemas" : { + "CTAS_INNER_JOIN_2_1.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_2_1.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_2_1.INNER_JOIN_2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "X", + "VALUE" : 0, + "F1" : "yo dawg", + "F2" : 50 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic_2", + "key" : 0, + "value" : { + "F3" : "I heard you like joins" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 100, + "value" : { + "NAME" : "X", + "VALUE" : 0, + "F1" : "KSQL has table-table joins", + "F2" : 50 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic_2", + "key" : 100, + "value" : { + "F3" : "so now you can join your join" + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN_2", + "key" : 0, + "value" : { + "NAME" : "X", + "F1" : "yo dawg", + "F3" : "I heard you like joins" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN_2", + "key" : 100, + "value" : { + "NAME" : "X", + "F1" : "KSQL has table-table joins", + "F3" : "so now you can join your join" + }, + "timestamp" : 20000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/topology new file mode 100644 index 000000000000..d6b0ba89f492 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_a_table_join_pipeline_-_JSON/6.0.0_1583319045585/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [INNER_JOIN]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic_2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: INNER_JOIN_2) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/plan.json new file mode 100644 index 000000000000..b182ddeb8d03 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/plan.json @@ -0,0 +1,253 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID INTEGER KEY, V BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON', WINDOW_TYPE='SESSION');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` INTEGER KEY, `V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + } + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID INTEGER KEY, V BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON', WINDOW_TYPE='SESSION');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` INTEGER KEY, `V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + } + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n S1.V S1_V,\n S2.V S2_V\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 1 MINUTES ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `S1_V` BIGINT, `S2_V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + } + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "windowedStreamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `V` BIGINT" + }, + "selectExpressions" : [ "V AS S1_V", "ROWTIME AS S1_ROWTIME", "ID AS S1_ID", "WINDOWSTART AS S1_WINDOWSTART", "WINDOWEND AS S1_WINDOWEND" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "windowedStreamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "SESSION", + "size" : null + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `V` BIGINT" + }, + "selectExpressions" : [ "V AS S2_V", "ROWTIME AS S2_ROWTIME", "ID AS S2_ID", "WINDOWSTART AS S2_WINDOWSTART", "WINDOWEND AS S2_WINDOWEND" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000 + }, + "selectExpressions" : [ "S1_V AS S1_V", "S2_V AS S2_V" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/spec.json new file mode 100644 index 000000000000..682d83596a99 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/spec.json @@ -0,0 +1,62 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046811, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "V" : 1 + }, + "timestamp" : 765, + "window" : { + "start" : 234, + "end" : 765, + "type" : "SESSION" + } + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "V" : 2 + }, + "timestamp" : 567, + "window" : { + "start" : 234, + "end" : 567, + "type" : "SESSION" + } + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "V" : 3 + }, + "timestamp" : 765, + "window" : { + "start" : 234, + "end" : 765, + "type" : "SESSION" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "S1_V" : 1, + "S2_V" : 3 + }, + "timestamp" : 765, + "window" : { + "start" : 234, + "end" : 765, + "type" : "SESSION" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/topology new file mode 100644 index 000000000000..72efdd20ee59 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_session_windowed/6.0.0_1583319046811/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/plan.json new file mode 100644 index 000000000000..d13cccc7b1a6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/plan.json @@ -0,0 +1,253 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID INTEGER KEY, V BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON', WINDOW_SIZE='5 SECONDS', WINDOW_TYPE='Hopping');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` INTEGER KEY, `V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID INTEGER KEY, V BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON', WINDOW_SIZE='2 SECOND', WINDOW_TYPE='Tumbling');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` INTEGER KEY, `V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 2.000000000 + } + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 1 MINUTES ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `S1_ROWTIME` BIGINT, `S1_ID` INTEGER, `S1_WINDOWSTART` BIGINT, `S1_WINDOWEND` BIGINT, `S1_V` BIGINT, `S2_ROWTIME` BIGINT, `S2_ID` INTEGER, `S2_WINDOWSTART` BIGINT, `S2_WINDOWEND` BIGINT, `S2_V` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + } + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "windowedStreamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "HOPPING", + "size" : 5.000000000 + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `V` BIGINT" + }, + "selectExpressions" : [ "V AS S1_V", "ROWTIME AS S1_ROWTIME", "ID AS S1_ID", "WINDOWSTART AS S1_WINDOWSTART", "WINDOWEND AS S1_WINDOWEND" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "windowedStreamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : { + "type" : "TUMBLING", + "size" : 2.000000000 + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `V` BIGINT" + }, + "selectExpressions" : [ "V AS S2_V", "ROWTIME AS S2_ROWTIME", "ID AS S2_ID", "WINDOWSTART AS S2_WINDOWSTART", "WINDOWEND AS S2_WINDOWEND" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000 + }, + "selectExpressions" : [ "S1_ROWTIME AS S1_ROWTIME", "S1_ID AS S1_ID", "S1_WINDOWSTART AS S1_WINDOWSTART", "S1_WINDOWEND AS S1_WINDOWEND", "S1_V AS S1_V", "S2_ROWTIME AS S2_ROWTIME", "S2_ID AS S2_ID", "S2_WINDOWSTART AS S2_WINDOWSTART", "S2_WINDOWEND AS S2_WINDOWEND", "S2_V AS S2_V" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/spec.json new file mode 100644 index 000000000000..f549179361ba --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/spec.json @@ -0,0 +1,115 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046881, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "V" : 1 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 5000, + "type" : "TIME" + } + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "V" : 2 + }, + "timestamp" : 1000, + "window" : { + "start" : 1000, + "end" : 6000, + "type" : "TIME" + } + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "V" : 3 + }, + "timestamp" : 2000, + "window" : { + "start" : 2000, + "end" : 7000, + "type" : "TIME" + } + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "V" : 4 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 2000, + "type" : "TIME" + } + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "V" : 5 + }, + "timestamp" : 2000, + "window" : { + "start" : 2000, + "end" : 4000, + "type" : "TIME" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "S1_ROWTIME" : 0, + "S1_WINDOWSTART" : 0, + "S1_WINDOWEND" : 5000, + "S1_ID" : 1, + "S1_V" : 1, + "S2_ROWTIME" : 0, + "S2_WINDOWSTART" : 0, + "S2_WINDOWEND" : 2000, + "S2_ID" : 1, + "S2_V" : 4 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 5000, + "type" : "TIME" + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "S1_ROWTIME" : 2000, + "S1_WINDOWSTART" : 2000, + "S1_WINDOWEND" : 7000, + "S1_ID" : 1, + "S1_V" : 3, + "S2_ROWTIME" : 2000, + "S2_WINDOWSTART" : 2000, + "S2_WINDOWEND" : 4000, + "S2_ID" : 1, + "S2_V" : 5 + }, + "timestamp" : 2000, + "window" : { + "start" : 2000, + "end" : 7000, + "type" : "TIME" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/topology new file mode 100644 index 000000000000..72efdd20ee59 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_matching_time_windowed/6.0.0_1583319046881/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/plan.json new file mode 100644 index 000000000000..366bc797fefe --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 BIGINT, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 BIGINT, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/spec.json new file mode 100644 index 000000000000..13d0253d6a95 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046422, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1000000000, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1000000000, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1000000000, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_AVRO/6.0.0_1583319046422/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/plan.json new file mode 100644 index 000000000000..3d520a59bb3d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 BIGINT, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 BIGINT, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/spec.json new file mode 100644 index 000000000000..ef89f3d51e60 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046460, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1000000000, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1000000000, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1000000000, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_JSON/6.0.0_1583319046460/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/plan.json new file mode 100644 index 000000000000..a3d6552d1297 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 BIGINT, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 BIGINT, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/spec.json new file mode 100644 index 000000000000..f98bf209d4e7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046499, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1000000000, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1000000000, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1000000000, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_BIGINT_column_-_PROTOBUF/6.0.0_1583319046499/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/plan.json new file mode 100644 index 000000000000..beee5c63235a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 DOUBLE, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 DOUBLE, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/spec.json new file mode 100644 index 000000000000..57fad92af725 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046539, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1.23, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1.23, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1.23, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_AVRO/6.0.0_1583319046539/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/plan.json new file mode 100644 index 000000000000..5d614523345e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 DOUBLE, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 DOUBLE, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/spec.json new file mode 100644 index 000000000000..eef2e8f4acb0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046589, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1.23, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1.23, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1.23, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_JSON/6.0.0_1583319046589/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/plan.json new file mode 100644 index 000000000000..658078877c95 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 DOUBLE, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 DOUBLE, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/spec.json new file mode 100644 index 000000000000..58d36f6ded5a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046631, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1.23, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1.23, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1.23, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_DOUBLE_column_-_PROTOBUF/6.0.0_1583319046631/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/plan.json new file mode 100644 index 000000000000..700388c67063 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 INTEGER, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 INTEGER, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/spec.json new file mode 100644 index 000000000000..ebe8db66c4f4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046311, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 10, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 10, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_AVRO/6.0.0_1583319046311/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/plan.json new file mode 100644 index 000000000000..cd897939e17d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 INTEGER, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 INTEGER, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/spec.json new file mode 100644 index 000000000000..eaa718c563df --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046347, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 10, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 10, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_JSON/6.0.0_1583319046347/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/plan.json new file mode 100644 index 000000000000..76d65db20064 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 INTEGER, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 INTEGER, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/spec.json new file mode 100644 index 000000000000..70139449bdab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046384, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 10, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 10, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_INT_column_-_PROTOBUF/6.0.0_1583319046384/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/plan.json new file mode 100644 index 000000000000..70fa2e1d6d00 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 STRING, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 STRING, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/spec.json new file mode 100644 index 000000000000..f92b27b97cb4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046673, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : "x", + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : "x", + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_AVRO/6.0.0_1583319046673/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/plan.json new file mode 100644 index 000000000000..5270c11ec3cd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 STRING, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 STRING, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/spec.json new file mode 100644 index 000000000000..23dfe677139c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046719, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : "x", + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : "x", + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_JSON/6.0.0_1583319046719/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/plan.json new file mode 100644 index 000000000000..66582e88d4cd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 STRING, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 STRING, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER" + }, + "keyExpression" : "L0" + }, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER" + }, + "keyExpression" : "R0" + }, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/spec.json new file mode 100644 index 000000000000..2b99ae70d061 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046763, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : "x", + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : "x", + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_KAFKA_STRING_column_-_PROTOBUF/6.0.0_1583319046763/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/plan.json new file mode 100644 index 000000000000..249e2fa4be48 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/plan.json @@ -0,0 +1,230 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT_STREAM (K STRING KEY, SF BIGINT) WITH (KAFKA_TOPIC='stream_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT_STREAM", + "schema" : "`K` STRING KEY, `SF` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "stream_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INPUT_TABLE (ID BIGINT KEY, TF INTEGER) WITH (KAFKA_TOPIC='table_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INPUT_TABLE", + "schema" : "`ID` BIGINT KEY, `TF` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "table_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT_STREAM S\nINNER JOIN INPUT_TABLE T ON ((S.SF = T.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` BIGINT KEY, `S_ROWTIME` BIGINT, `S_K` STRING, `S_SF` BIGINT, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_TF` INTEGER", + "keyField" : "S_SF", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT_STREAM", "INPUT_TABLE" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "stream_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `SF` BIGINT" + }, + "keyExpression" : "SF" + }, + "selectExpressions" : [ "SF AS S_SF", "ROWTIME AS S_ROWTIME", "K AS S_K" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "table_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `TF` INTEGER" + }, + "selectExpressions" : [ "TF AS T_TF", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + } + }, + "selectExpressions" : [ "S_ROWTIME AS S_ROWTIME", "S_K AS S_K", "S_SF AS S_SF", "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_TF AS T_TF" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/spec.json new file mode 100644 index 000000000000..e6ccb91b3ec6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046266, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "table_topic", + "key" : 26589, + "value" : { + "TF" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "stream_topic", + "key" : "a", + "value" : { + "SF" : 12589 + }, + "timestamp" : 100 + }, { + "topic" : "table_topic", + "key" : 12589, + "value" : { + "TF" : 12 + }, + "timestamp" : 200 + }, { + "topic" : "stream_topic", + "key" : "b", + "value" : { + "SF" : 12589 + }, + "timestamp" : 300 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 12589, + "value" : { + "S_K" : "b", + "S_ROWTIME" : 300, + "S_SF" : 12589, + "T_ROWTIME" : 300, + "T_ID" : 12589, + "T_TF" : 12 + }, + "timestamp" : 300 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/topology new file mode 100644 index 000000000000..e077a6406310 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_non_STRING_value_column/6.0.0_1583319046266/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: Join-repartition-source (topics: [Join-repartition]) + --> Join + Source: KSTREAM-SOURCE-0000000000 (topics: [table_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- Join-repartition-source + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Join + Sink: KSTREAM-SINK-0000000014 (topic: OUTPUT) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [stream_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> KSTREAM-FILTER-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KSTREAM-FILTER-0000000006 (stores: []) + --> KSTREAM-KEY-SELECT-0000000007 + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-KEY-SELECT-0000000007 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000006 + Processor: PrependAliasLeft (stores: []) + --> Join-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000007 + Processor: Join-repartition-filter (stores: []) + --> Join-repartition-sink + <-- PrependAliasLeft + Sink: Join-repartition-sink (topic: Join-repartition) + <-- Join-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/plan.json new file mode 100644 index 000000000000..bf83e940e860 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (LEFT_ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`LEFT_ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (RIGHT_ID BIGINT KEY, F1 STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`RIGHT_ID` BIGINT KEY, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.NAME NAME,\n TT.F1 F1\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.LEFT_ID = TT.RIGHT_ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`LEFT_ID` BIGINT KEY, `NAME` STRING, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`LEFT_ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "LEFT_ID AS T_LEFT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`RIGHT_ID` BIGINT KEY, `F1` STRING" + }, + "selectExpressions" : [ "F1 AS TT_F1", "ROWTIME AS TT_ROWTIME", "RIGHT_ID AS TT_RIGHT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "TT_F1 AS F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/spec.json new file mode 100644 index 000000000000..b0697a888752 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/spec.json @@ -0,0 +1,125 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046239, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100" + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah" + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety" + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar" + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "F1" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "NAME" : "100", + "F1" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "F1" : "blah" + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "F1" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "F1" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/topology new file mode 100644 index 000000000000..c9d348c479af --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_on_unqualified_join_criteria/6.0.0_1583319046239/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/plan.json new file mode 100644 index 000000000000..41e395aa9bdd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/spec.json new file mode 100644 index 000000000000..2160168926a0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043960, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_AVRO/6.0.0_1583319043960/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/plan.json new file mode 100644 index 000000000000..4419ed6f0d80 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/spec.json new file mode 100644 index 000000000000..75a77a90dd72 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044009, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_JSON/6.0.0_1583319044009/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/plan.json new file mode 100644 index 000000000000..666d25f892b2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/spec.json new file mode 100644 index 000000000000..ced8c161cf69 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/spec.json @@ -0,0 +1,107 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044058, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_-_PROTOBUF/6.0.0_1583319044058/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/plan.json new file mode 100644 index 000000000000..cda2f1f95839 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT *\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `TT_ROWTIME` BIGINT, `TT_ID` BIGINT, `TT_NAME` STRING, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING" + }, + "selectExpressions" : [ "F1 AS T_F1", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "TT_ROWTIME AS TT_ROWTIME", "TT_ID AS TT_ID", "TT_NAME AS TT_NAME", "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/spec.json new file mode 100644 index 000000000000..426aa70ceab9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/spec.json @@ -0,0 +1,217 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044370, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100" + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah" + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety" + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar" + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 0, + "TT_ID" : 0, + "TT_NAME" : "zero" + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 10000, + "T_ID" : 0, + "T_F1" : "blah" + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "TT_ROWTIME" : 11000, + "TT_ID" : 10, + "TT_NAME" : "100" + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 13000, + "TT_ID" : 0, + "TT_NAME" : "foo" + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 15000, + "T_ID" : 0, + "T_F1" : "a" + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "T_ROWTIME" : 16000, + "T_ID" : 100, + "T_F1" : "newblah" + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "TT_ROWTIME" : 17000, + "TT_ID" : 90, + "TT_NAME" : "ninety" + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 30000, + "TT_ID" : 0, + "TT_NAME" : "bar" + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "zero", + "TT_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_AVRO/6.0.0_1583319044370/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/plan.json new file mode 100644 index 000000000000..5e8d59abdf30 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT *\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `TT_ROWTIME` BIGINT, `TT_ID` BIGINT, `TT_NAME` STRING, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING" + }, + "selectExpressions" : [ "F1 AS T_F1", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "TT_ROWTIME AS TT_ROWTIME", "TT_ID AS TT_ID", "TT_NAME AS TT_NAME", "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/spec.json new file mode 100644 index 000000000000..10391d2cad5c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/spec.json @@ -0,0 +1,217 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044415, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100" + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah" + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety" + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar" + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 0, + "TT_ID" : 0, + "TT_NAME" : "zero" + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 10000, + "T_ID" : 0, + "T_F1" : "blah" + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "TT_ROWTIME" : 11000, + "TT_ID" : 10, + "TT_NAME" : "100" + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 13000, + "TT_ID" : 0, + "TT_NAME" : "foo" + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 15000, + "T_ID" : 0, + "T_F1" : "a" + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "T_ROWTIME" : 16000, + "T_ID" : 100, + "T_F1" : "newblah" + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "TT_ROWTIME" : 17000, + "TT_ID" : 90, + "TT_NAME" : "ninety" + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 30000, + "TT_ID" : 0, + "TT_NAME" : "bar" + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "zero", + "TT_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_JSON/6.0.0_1583319044415/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/plan.json new file mode 100644 index 000000000000..37800802148f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT *\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `TT_ROWTIME` BIGINT, `TT_ID` BIGINT, `TT_NAME` STRING, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING" + }, + "selectExpressions" : [ "F1 AS T_F1", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "TT_ROWTIME AS TT_ROWTIME", "TT_ID AS TT_ID", "TT_NAME AS TT_NAME", "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/spec.json new file mode 100644 index 000000000000..fe6de52a5873 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/spec.json @@ -0,0 +1,217 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044454, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100" + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah" + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety" + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar" + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 0, + "TT_ID" : 0, + "TT_NAME" : "zero" + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 10000, + "T_ID" : 0, + "T_F1" : "blah" + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "TT_ROWTIME" : 11000, + "TT_ID" : 10, + "TT_NAME" : "100" + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 13000, + "TT_ID" : 0, + "TT_NAME" : "foo" + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 15000, + "T_ID" : 0, + "T_F1" : "a" + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "T_ROWTIME" : 16000, + "T_ID" : 100, + "T_F1" : "newblah" + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "TT_ROWTIME" : 17000, + "TT_ID" : 90, + "TT_NAME" : "ninety" + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 30000, + "TT_ID" : 0, + "TT_NAME" : "bar" + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "zero", + "TT_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_NAME" : "foo", + "TT_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_fields_-_PROTOBUF/6.0.0_1583319044454/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/plan.json new file mode 100644 index 000000000000..442b77632e82 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.F1 F1\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_NAME AS T_NAME", "T_VALUE AS T_VALUE", "TT_F1 AS F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/spec.json new file mode 100644 index 000000000000..2227fc2c1013 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044111, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0, + "F1" : "blah", + "T_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "blah", + "T_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "a", + "T_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_AVRO/6.0.0_1583319044111/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/plan.json new file mode 100644 index 000000000000..55c5d8a1b647 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.F1 F1\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_NAME AS T_NAME", "T_VALUE AS T_VALUE", "TT_F1 AS F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/spec.json new file mode 100644 index 000000000000..8f3f33665743 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044151, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0, + "F1" : "blah", + "T_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "blah", + "T_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "a", + "T_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_JSON/6.0.0_1583319044151/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/plan.json new file mode 100644 index 000000000000..a8bb740c7672 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.F1 F1\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_NAME AS T_NAME", "T_VALUE AS T_VALUE", "TT_F1 AS F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/spec.json new file mode 100644 index 000000000000..7fd90199af52 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044195, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0, + "F1" : "blah", + "T_ROWTIME" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "blah", + "T_ROWTIME" : 13000 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "a", + "T_ROWTIME" : 13000 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_left_fields_some_right_-_PROTOBUF/6.0.0_1583319044195/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/plan.json new file mode 100644 index 000000000000..3f5c86958b2d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.NAME NAME\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "VALUE AS TT_VALUE", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS T_F1", "F2 AS T_F2", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1", "T_F2 AS T_F2", "TT_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/spec.json new file mode 100644 index 000000000000..e16a4829fc2d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044242, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "zero" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_F2" : 10, + "T_ROWTIME" : 15000, + "NAME" : "foo" + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_AVRO/6.0.0_1583319044242/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/plan.json new file mode 100644 index 000000000000..dafee028fdae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.NAME NAME\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "VALUE AS TT_VALUE", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS T_F1", "F2 AS T_F2", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1", "T_F2 AS T_F2", "TT_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/spec.json new file mode 100644 index 000000000000..3dd231f16062 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044289, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "zero" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_F2" : 10, + "T_ROWTIME" : 15000, + "NAME" : "foo" + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_JSON/6.0.0_1583319044289/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/plan.json new file mode 100644 index 000000000000..bca7c18e0676 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.NAME NAME\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ROWTIME` BIGINT, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "VALUE AS TT_VALUE", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS T_F1", "F2 AS T_F2", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ROWTIME AS T_ROWTIME", "T_ID AS T_ID", "T_F1 AS T_F1", "T_F2 AS T_F2", "TT_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/spec.json new file mode 100644 index 000000000000..70bc2351cf12 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044330, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "zero" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "T_ROWTIME" : 10000, + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_F2" : 10, + "T_ROWTIME" : 15000, + "NAME" : "foo" + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_all_right_fields_some_left_-_PROTOBUF/6.0.0_1583319044330/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/plan.json new file mode 100644 index 000000000000..c10749b40aa1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN (11 SECONDS, 10 SECONDS) ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/spec.json new file mode 100644 index 000000000000..521b5dbbd6be --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/spec.json @@ -0,0 +1,99 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044492, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 12000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_AVRO/6.0.0_1583319044492/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/plan.json new file mode 100644 index 000000000000..9f44c5e232a1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN (11 SECONDS, 10 SECONDS) ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/spec.json new file mode 100644 index 000000000000..8f175377c90e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/spec.json @@ -0,0 +1,99 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044534, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 12000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_JSON/6.0.0_1583319044534/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/plan.json new file mode 100644 index 000000000000..ee68ba3797a9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN (11 SECONDS, 10 SECONDS) ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/spec.json new file mode 100644 index 000000000000..7d5b78b4a086 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/spec.json @@ -0,0 +1,99 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044577, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 12000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF/6.0.0_1583319044577/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/plan.json new file mode 100644 index 000000000000..fe251abe9fba --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 10 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/spec.json new file mode 100644 index 000000000000..0dd90bdb671b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/spec.json @@ -0,0 +1,140 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044623, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000 + }, + "timestamp" : 6000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_AVRO/6.0.0_1583319044623/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/plan.json new file mode 100644 index 000000000000..7bdd65027f10 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 10 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/spec.json new file mode 100644 index 000000000000..8b91abbab0fa --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/spec.json @@ -0,0 +1,140 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044669, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000 + }, + "timestamp" : 6000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_JSON/6.0.0_1583319044669/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/plan.json new file mode 100644 index 000000000000..128ed803162b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 10 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/spec.json new file mode 100644 index 000000000000..23b7e99e47db --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/spec.json @@ -0,0 +1,140 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044712, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000 + }, + "timestamp" : 6000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_inner_join_with_out_of_order_messages_-_PROTOBUF/6.0.0_1583319044712/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/plan.json new file mode 100644 index 000000000000..f9874bcd0cdb --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID INTEGER KEY, IGNORED STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` INTEGER KEY, `IGNORED` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID INTEGER KEY, IGNORED STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` INTEGER KEY, `IGNORED` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT T.ID T_ID\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = (CASE WHEN (TT.ID = 2) THEN 1 ELSE 3 END)))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `T_ID` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "IGNORED AS T_IGNORED", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` STRING" + }, + "keyExpression" : "(CASE WHEN (ID = 2) THEN 1 ELSE 3 END)" + }, + "selectExpressions" : [ "IGNORED AS TT_IGNORED", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/spec.json new file mode 100644 index 000000000000..8b3e74668aa8 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/spec.json @@ -0,0 +1,35 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046181, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { }, + "timestamp" : 0 + }, { + "topic" : "left_topic", + "key" : 3, + "value" : { }, + "timestamp" : 5 + }, { + "topic" : "right_topic", + "key" : 2, + "value" : { }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_ID" : 1 + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CASE_expression/6.0.0_1583319046181/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/plan.json new file mode 100644 index 000000000000..27fd5ff22777 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID BIGINT KEY, X BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` BIGINT KEY, `X` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID INTEGER KEY, X INTEGER) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` INTEGER KEY, `X` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT T.X T_X\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = CAST(TT.ID AS BIGINT)))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `T_X` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `X` BIGINT" + }, + "selectExpressions" : [ "X AS T_X", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `X` INTEGER" + }, + "keyExpression" : "CAST(ID AS BIGINT)" + }, + "selectExpressions" : [ "X AS TT_X", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_X AS T_X" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/spec.json new file mode 100644 index 000000000000..f21dde5873a4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/spec.json @@ -0,0 +1,43 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046030, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "x" : 2 + }, + "timestamp" : 10 + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "x" : 3 + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "key" : 1, + "value" : { + "TT_X" : 3, + "TT_ROWTIME" : 10, + "TT_ID" : 1 + }, + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_X" : 2 + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST/6.0.0_1583319046030/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/plan.json new file mode 100644 index 000000000000..ffdf276fbd51 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID INTEGER KEY, X BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` INTEGER KEY, `X` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID DOUBLE KEY, X BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` DOUBLE KEY, `X` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT L.X L_X\nFROM L L\nINNER JOIN R R WITHIN 30 SECONDS ON ((L.ID = CAST(R.ID AS INTEGER)))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `L_X` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `X` BIGINT" + }, + "selectExpressions" : [ "X AS L_X", "ROWTIME AS L_ROWTIME", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` DOUBLE KEY, `X` BIGINT" + }, + "keyExpression" : "CAST(ID AS INTEGER)" + }, + "selectExpressions" : [ "X AS R_X", "ROWTIME AS R_ROWTIME", "ID AS R_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "L_X AS L_X" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/spec.json new file mode 100644 index 000000000000..dbf8cc21a676 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/spec.json @@ -0,0 +1,34 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046064, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "x" : 2 + }, + "timestamp" : 10 + }, { + "topic" : "right_topic", + "key" : 1.0, + "value" : { + "x" : 3 + }, + "timestamp" : 11 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "L_X" : 2 + }, + "timestamp" : 11 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_CAST_double_to_int/6.0.0_1583319046064/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/plan.json new file mode 100644 index 000000000000..51f943b04b1b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n TT.ID TT_ID\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = (TT.ID + 1)))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `T_ID` INTEGER, `TT_ID` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "keyExpression" : "(ID + 1)" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "TT_ID AS TT_ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/spec.json new file mode 100644 index 000000000000..47c618542fc4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/spec.json @@ -0,0 +1,35 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046120, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "name" : "-" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "name" : "-" + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 0 + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_binary_expression/6.0.0_1583319046120/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/plan.json new file mode 100644 index 000000000000..f283b2ac4bab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.NAME T_NAME,\n TT.NAME TT_NAME\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = -TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `T_NAME` STRING, `TT_NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "keyExpression" : "-ID" + }, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_NAME AS T_NAME", "TT_NAME AS TT_NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/spec.json new file mode 100644 index 000000000000..da35150aa7f3 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/spec.json @@ -0,0 +1,35 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046150, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "name" : "a" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : -1, + "value" : { + "name" : "b" + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_NAME" : "a", + "TT_NAME" : "b" + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression/6.0.0_1583319046150/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/plan.json new file mode 100644 index 000000000000..1f81b9a690e9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID INTEGER KEY, IGNORED STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` INTEGER KEY, `IGNORED` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID INTEGER KEY, IGNORED STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` INTEGER KEY, `IGNORED` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT T.ID T_ID\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((-TT.ID = T.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `T_ID` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` STRING" + }, + "selectExpressions" : [ "IGNORED AS T_IGNORED", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `IGNORED` STRING" + }, + "keyExpression" : "-ID" + }, + "selectExpressions" : [ "IGNORED AS TT_IGNORED", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/spec.json new file mode 100644 index 000000000000..8783ce3cd339 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/spec.json @@ -0,0 +1,30 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046211, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : -1, + "value" : { }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_ID" : 1 + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_arithmetic_unary_expression_flipped_sides/6.0.0_1583319046211/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/plan.json new file mode 100644 index 000000000000..9c85b04cc005 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ROWKEY` STRING KEY, `ID` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (ID STRING) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ROWKEY` STRING KEY, `ID` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT T.ID T_ID\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = SUBSTRING(TT.ID, 2)))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `T_ID` STRING", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` STRING" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "ROWTIME AS T_ROWTIME", "ROWKEY AS T_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` STRING" + }, + "keyExpression" : "SUBSTRING(ID, 2)" + }, + "selectExpressions" : [ "ID AS TT_ID", "ROWTIME AS TT_ROWTIME", "ROWKEY AS TT_ROWKEY" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/spec.json new file mode 100644 index 000000000000..71d72c2455a1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/spec.json @@ -0,0 +1,34 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046001, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "id" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "!foo", + "value" : { + "id" : "!foo" + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "foo", + "value" : { + "T_ID" : "foo" + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/topology new file mode 100644 index 000000000000..239bf5b7d656 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_function/6.0.0_1583319046001/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/plan.json new file mode 100644 index 000000000000..a68e562930dd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/plan.json @@ -0,0 +1,243 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST1 (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST1", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 (K STRING KEY, ID ARRAY) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`K` STRING KEY, `ID` ARRAY", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT T.ID T_ID\nFROM TEST1 T\nINNER JOIN TEST2 TT WITHIN 30 SECONDS ON ((T.ID = TT.ID[1]))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `T_ID` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST1", "TEST2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` ARRAY" + }, + "keyExpression" : "ID[1]" + }, + "selectExpressions" : [ "ID AS TT_ID", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 30.000000000, + "afterMillis" : 30.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/spec.json new file mode 100644 index 000000000000..c5c7ec2d6513 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/spec.json @@ -0,0 +1,41 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319046093, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT> NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT, TT_ROWTIME BIGINT, TT_K VARCHAR> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "name" : "-" + }, + "timestamp" : 0 + }, { + "topic" : "left_topic", + "key" : 2, + "value" : { + "name" : "-" + }, + "timestamp" : 5 + }, { + "topic" : "right_topic", + "key" : "k", + "value" : { + "id" : [ 1, 2, 3 ] + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "T_ID" : 1 + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/topology new file mode 100644 index 000000000000..7bfaa41645ab --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_join_-_contains_subscript/6.0.0_1583319046093/topology @@ -0,0 +1,54 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000013-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000014-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000017 + <-- Join-merge + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- Project + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> KSTREAM-FILTER-0000000005 + <-- KSTREAM-SOURCE-0000000003 + Processor: KSTREAM-FILTER-0000000005 (stores: []) + --> KSTREAM-KEY-SELECT-0000000006 + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-KEY-SELECT-0000000006 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000006 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/plan.json new file mode 100644 index 000000000000..f127ad6e517c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/spec.json new file mode 100644 index 000000000000..0ac03af43524 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/spec.json @@ -0,0 +1,267 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043519, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/topology new file mode 100644 index 000000000000..c9d348c479af --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_AVRO/6.0.0_1583319043519/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/plan.json new file mode 100644 index 000000000000..fd6c1dc28aff --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/spec.json new file mode 100644 index 000000000000..7b6072937a77 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/spec.json @@ -0,0 +1,267 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043567, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/topology new file mode 100644 index 000000000000..c9d348c479af --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_JSON/6.0.0_1583319043567/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/plan.json new file mode 100644 index 000000000000..d15174ce049c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/spec.json new file mode 100644 index 000000000000..2a553a2b0bf7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/spec.json @@ -0,0 +1,267 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043603, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/topology new file mode 100644 index 000000000000..c9d348c479af --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_PROTOBUF/6.0.0_1583319043603/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/plan.json new file mode 100644 index 000000000000..225da0a5400d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/spec.json new file mode 100644 index 000000000000..86d86897329c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/spec.json @@ -0,0 +1,378 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043796, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_AVRO/6.0.0_1583319043796/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/plan.json new file mode 100644 index 000000000000..131e7ad8849d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/spec.json new file mode 100644 index 000000000000..c6d2e1b21528 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/spec.json @@ -0,0 +1,378 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043850, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_JSON/6.0.0_1583319043850/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/plan.json new file mode 100644 index 000000000000..717985e503b3 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/spec.json new file mode 100644 index 000000000000..50a4a2a42c18 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/spec.json @@ -0,0 +1,378 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043903, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-right-repartition", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-Join-left-repartition", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 0, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "zero", + "T_VALUE" : 0 + }, + "timestamp" : 0, + "window" : { + "start" : 0, + "end" : 11000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 10000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "blah", + "TT_F2" : 50 + }, + "timestamp" : 10000, + "window" : { + "start" : 10000, + "end" : 21000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 10, + "value" : { + "T_ROWTIME" : 11000, + "T_K" : "", + "T_ID" : 10, + "T_NAME" : "100", + "T_VALUE" : 5 + }, + "timestamp" : 11000, + "window" : { + "start" : 11000, + "end" : 22000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 13000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "foo", + "T_VALUE" : 100 + }, + "timestamp" : 13000, + "window" : { + "start" : 13000, + "end" : 24000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 0, + "value" : { + "TT_ROWTIME" : 15000, + "TT_K" : "", + "TT_ID" : 0, + "TT_F1" : "a", + "TT_F2" : 10 + }, + "timestamp" : 15000, + "window" : { + "start" : 15000, + "end" : 26000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-OUTEROTHER-0000000019-store-changelog", + "key" : 100, + "value" : { + "TT_ROWTIME" : 16000, + "TT_K" : "", + "TT_ID" : 100, + "TT_F1" : "newblah", + "TT_F2" : 150 + }, + "timestamp" : 16000, + "window" : { + "start" : 16000, + "end" : 27000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 90, + "value" : { + "T_ROWTIME" : 17000, + "T_K" : "", + "T_ID" : 90, + "T_NAME" : "ninety", + "T_VALUE" : 90 + }, + "timestamp" : 17000, + "window" : { + "start" : 17000, + "end" : 28000, + "type" : "TIME" + } + }, { + "topic" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_OUTER_JOIN_0-KSTREAM-JOINTHIS-0000000018-store-changelog", + "key" : 0, + "value" : { + "T_ROWTIME" : 30000, + "T_K" : "", + "T_ID" : 0, + "T_NAME" : "bar", + "T_VALUE" : 99 + }, + "timestamp" : 30000, + "window" : { + "start" : 30000, + "end" : 41000, + "type" : "TIME" + } + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_-_rekey_-_PROTOBUF/6.0.0_1583319043903/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/plan.json new file mode 100644 index 000000000000..225da0a5400d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/spec.json new file mode 100644 index 000000000000..f592de6f3f58 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043641, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_AVRO/6.0.0_1583319043641/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/plan.json new file mode 100644 index 000000000000..131e7ad8849d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/spec.json new file mode 100644 index 000000000000..786f08268145 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043696, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_JSON/6.0.0_1583319043696/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/plan.json new file mode 100644 index 000000000000..b69d444b12ef --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/plan.json @@ -0,0 +1,250 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.K T_K,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ROWKEY` BIGINT KEY, `T_ID` BIGINT, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : "T_ID", + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV1", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT" + }, + "keyExpression" : "ID" + }, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_K AS T_K", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/spec.json new file mode 100644 index 000000000000..12e599e9689e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319043742, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_K" : "foo", + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_K" : "foo", + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "T_K" : "foo", + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "T_K" : "foo", + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_K" : "foo", + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/topology new file mode 100644 index 000000000000..1ce75cc61bdc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_left_join_with_rowkey_-_rekey_-_PROTOBUF/6.0.0_1583319043742/topology @@ -0,0 +1,69 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> KSTREAM-FILTER-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-FILTER-0000000002 (stores: []) + --> KSTREAM-KEY-SELECT-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-KEY-SELECT-0000000003 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-FILTER-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000003 + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000005 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-TRANSFORMVALUES-0000000006 (stores: []) + --> KSTREAM-FILTER-0000000007 + <-- KSTREAM-SOURCE-0000000005 + Processor: KSTREAM-FILTER-0000000007 (stores: []) + --> KSTREAM-KEY-SELECT-0000000008 + <-- KSTREAM-TRANSFORMVALUES-0000000006 + Processor: KSTREAM-KEY-SELECT-0000000008 (stores: []) + --> PrependAliasRight + <-- KSTREAM-FILTER-0000000007 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- KSTREAM-KEY-SELECT-0000000008 + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000018-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000019-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000022 + <-- Join-merge + Sink: KSTREAM-SINK-0000000022 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/plan.json new file mode 100644 index 000000000000..7e0f019f7597 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/spec.json new file mode 100644 index 000000000000..b594523c8d0f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/spec.json @@ -0,0 +1,165 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044758, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 100, + "value" : { + "T_ID" : null, + "NAME" : null, + "VALUE" : null, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/topology new file mode 100644 index 000000000000..fc9b64516970 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_AVRO/6.0.0_1583319044758/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-outer-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-outer-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-outer-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/plan.json new file mode 100644 index 000000000000..12a8bcf1f9e9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/spec.json new file mode 100644 index 000000000000..aa0163cc5d95 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/spec.json @@ -0,0 +1,165 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044802, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : null, + "F2" : null + }, + "timestamp" : 30000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 100, + "value" : { + "T_ID" : null, + "NAME" : null, + "VALUE" : null, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/topology new file mode 100644 index 000000000000..fc9b64516970 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_JSON/6.0.0_1583319044802/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-outer-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-outer-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-outer-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/plan.json new file mode 100644 index 000000000000..1cebaad582bc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_STREAM", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/spec.json new file mode 100644 index 000000000000..3375a09f55b7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/spec.json @@ -0,0 +1,165 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044848, + "schemas" : { + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 100, + "value" : { + "T_ID" : 0, + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/topology new file mode 100644 index 000000000000..fc9b64516970 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_outer_join_-_PROTOBUF/6.0.0_1583319044848/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-outer-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-outer-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-outer-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: LEFT_OUTER_JOIN) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/plan.json new file mode 100644 index 000000000000..e9e8e53966f1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S1', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S2', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n S1.NAME NAME1,\n S2.NAME NAME2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 1 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S1_NAME", "ROWTIME AS S1_ROWTIME", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S2_NAME", "ROWTIME AS S2_ROWTIME", "ID AS S2_ID" ] + }, + "beforeMillis" : 1.000000000, + "afterMillis" : 1.000000000 + }, + "selectExpressions" : [ "S1_NAME AS NAME1", "S2_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/spec.json new file mode 100644 index 000000000000..cf6a4e8007e1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/spec.json @@ -0,0 +1,41 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045735, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "S1", + "key" : 0, + "value" : "a", + "timestamp" : 0 + }, { + "topic" : "S2", + "key" : 0, + "value" : "b", + "timestamp" : 10 + }, { + "topic" : "S1", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "S2", + "key" : 0, + "value" : null, + "timestamp" : 30 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/topology new file mode 100644 index 000000000000..c3c7777d7bae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045735/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [S1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [S2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/plan.json new file mode 100644 index 000000000000..fc529fda1169 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S1', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S2', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT S1.NAME NAME\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 1 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S1_NAME", "ROWTIME AS S1_ROWTIME", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S2_NAME", "ROWTIME AS S2_ROWTIME", "ID AS S2_ID" ] + }, + "beforeMillis" : 1.000000000, + "afterMillis" : 1.000000000 + }, + "selectExpressions" : [ "S1_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/spec.json new file mode 100644 index 000000000000..41fd83761a1e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045768, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "VARCHAR" + }, + "inputs" : [ { + "topic" : "S1", + "key" : 0, + "value" : "a", + "timestamp" : 0 + }, { + "topic" : "S2", + "key" : 0, + "value" : "b", + "timestamp" : 10 + }, { + "topic" : "S1", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "S2", + "key" : 0, + "value" : null, + "timestamp" : 30 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : "a", + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/topology new file mode 100644 index 000000000000..c3c7777d7bae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045768/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [S1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [S2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/plan.json new file mode 100644 index 000000000000..172c6d7758fa --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n S1.NAME NAME1,\n S2.NAME NAME2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 1 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S1_NAME", "ROWTIME AS S1_ROWTIME", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S2_NAME", "ROWTIME AS S2_ROWTIME", "ID AS S2_ID" ] + }, + "beforeMillis" : 1.000000000, + "afterMillis" : 1.000000000 + }, + "selectExpressions" : [ "S1_NAME AS NAME1", "S2_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/spec.json new file mode 100644 index 000000000000..237b58382734 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/spec.json @@ -0,0 +1,73 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045708, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "S1", + "key" : 0, + "value" : { + "NAME" : "a" + }, + "timestamp" : 0 + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "NAME" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "S1", + "key" : 0, + "value" : { + "ID" : null + }, + "timestamp" : 20 + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "ID" : null + }, + "timestamp" : 30 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : "b" + }, + "timestamp" : 20 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : null + }, + "timestamp" : 30 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : null + }, + "timestamp" : 30 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/topology new file mode 100644 index 000000000000..c3c7777d7bae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-stream_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045708/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [S1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [S2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/plan.json new file mode 100644 index 000000000000..2752a2c24ffc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/spec.json new file mode 100644 index 000000000000..7cd1e1e6e912 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/spec.json @@ -0,0 +1,101 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045441, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/topology new file mode 100644 index 000000000000..5e7df78be8d9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_AVRO/6.0.0_1583319045441/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: INNER_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/plan.json new file mode 100644 index 000000000000..b23e9d6692bb --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/spec.json new file mode 100644 index 000000000000..d5023984879b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/spec.json @@ -0,0 +1,101 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045487, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/topology new file mode 100644 index 000000000000..5e7df78be8d9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_JSON/6.0.0_1583319045487/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: INNER_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/plan.json new file mode 100644 index 000000000000..a8c74c902fcd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/spec.json new file mode 100644 index 000000000000..b2b28b965865 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/spec.json @@ -0,0 +1,101 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045537, + "schemas" : { + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/topology new file mode 100644 index 000000000000..5e7df78be8d9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_inner_join_-_PROTOBUF/6.0.0_1583319045537/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: INNER_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/plan.json new file mode 100644 index 000000000000..990ec10de7b0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/spec.json new file mode 100644 index 000000000000..75d766bfb1e9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/spec.json @@ -0,0 +1,112 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045300, + "schemas" : { + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.LEFT_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/topology new file mode 100644 index 000000000000..c5bc83246d3f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_AVRO/6.0.0_1583319045300/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: LEFT_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/plan.json new file mode 100644 index 000000000000..e350e1566ccd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/spec.json new file mode 100644 index 000000000000..b717b40795dc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/spec.json @@ -0,0 +1,112 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045348, + "schemas" : { + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.LEFT_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/topology new file mode 100644 index 000000000000..c5bc83246d3f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_JSON/6.0.0_1583319045348/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: LEFT_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/plan.json new file mode 100644 index 000000000000..e34d1a17373d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/spec.json new file mode 100644 index 000000000000..9f53c83a5550 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/spec.json @@ -0,0 +1,112 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045397, + "schemas" : { + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_LEFT_JOIN_0.LEFT_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 15000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/topology new file mode 100644 index 000000000000..c5bc83246d3f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_left_join_-_PROTOBUF/6.0.0_1583319045397/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_table]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: LEFT_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/plan.json new file mode 100644 index 000000000000..d005c8a19651 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n S.NAME NAME1,\n T.NAME NAME2\nFROM S S\nINNER JOIN T T ON ((S.ID = T.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S", "T" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S_NAME", "ROWTIME AS S_ROWTIME", "ID AS S_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + } + }, + "selectExpressions" : [ "S_NAME AS NAME1", "T_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/spec.json new file mode 100644 index 000000000000..d2bb425cd8d8 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/spec.json @@ -0,0 +1,45 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045835, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "T", + "key" : 0, + "value" : "b", + "timestamp" : 0 + }, { + "topic" : "S", + "key" : 0, + "value" : "a", + "timestamp" : 10 + }, { + "topic" : "S", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "T", + "key" : 0, + "value" : null, + "timestamp" : 30 + }, { + "topic" : "S", + "key" : 0, + "value" : null, + "timestamp" : 40 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/topology new file mode 100644 index 000000000000..f16734d3f43b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045835/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [S]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [T]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: OUTPUT) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/plan.json new file mode 100644 index 000000000000..f5fb9794fe73 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT S.NAME NAME\nFROM S S\nINNER JOIN T T ON ((S.ID = T.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S", "T" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S_NAME", "ROWTIME AS S_ROWTIME", "ID AS S_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + } + }, + "selectExpressions" : [ "S_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/spec.json new file mode 100644 index 000000000000..54a65e5e952c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/spec.json @@ -0,0 +1,42 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045871, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "VARCHAR" + }, + "inputs" : [ { + "topic" : "T", + "key" : 0, + "value" : "b", + "timestamp" : 0 + }, { + "topic" : "S", + "key" : 0, + "value" : "a", + "timestamp" : 10 + }, { + "topic" : "S", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "T", + "key" : 0, + "value" : null, + "timestamp" : 30 + }, { + "topic" : "S", + "key" : 0, + "value" : null, + "timestamp" : 40 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : "a", + "timestamp" : 10 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/topology new file mode 100644 index 000000000000..f16734d3f43b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045871/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [S]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [T]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: OUTPUT) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/plan.json new file mode 100644 index 000000000000..3b7869211c36 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='S', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n S.NAME NAME1,\n T.NAME NAME2\nFROM S S\nINNER JOIN T T ON ((S.ID = T.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S", "T" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "S", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS S_NAME", "ROWTIME AS S_ROWTIME", "ID AS S_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T_NAME", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + } + }, + "selectExpressions" : [ "S_NAME AS NAME1", "T_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/spec.json new file mode 100644 index 000000000000..bced2e929c00 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/spec.json @@ -0,0 +1,83 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045798, + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "T", + "key" : 0, + "value" : { + "NAME" : "b" + }, + "timestamp" : 0 + }, { + "topic" : "S", + "key" : 0, + "value" : { + "NAME" : "a" + }, + "timestamp" : 10 + }, { + "topic" : "S", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 20 + }, { + "topic" : "T", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 30 + }, { + "topic" : "S", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 40 + }, { + "topic" : "T", + "key" : 0, + "value" : null, + "timestamp" : 50 + }, { + "topic" : "S", + "key" : 0, + "value" : { + "NAME" : "a" + }, + "timestamp" : 60 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : "b" + }, + "timestamp" : 20 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : null + }, + "timestamp" : 40 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/topology new file mode 100644 index 000000000000..f16734d3f43b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_stream-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045798/topology @@ -0,0 +1,30 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [S]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Source: KSTREAM-SOURCE-0000000000 (topics: [T]) + --> KTABLE-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Join + Sink: KSTREAM-SINK-0000000009 (topic: OUTPUT) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/plan.json new file mode 100644 index 000000000000..fb9dedb32c80 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/spec.json new file mode 100644 index 000000000000..c3d0efdfc9ae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/spec.json @@ -0,0 +1,119 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045047, + "schemas" : { + "CTAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/topology new file mode 100644 index 000000000000..470702b9f891 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_AVRO/6.0.0_1583319045047/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: INNER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/plan.json new file mode 100644 index 000000000000..871714131776 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/spec.json new file mode 100644 index 000000000000..a245b03a6bbd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/spec.json @@ -0,0 +1,119 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045090, + "schemas" : { + "CTAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/topology new file mode 100644 index 000000000000..470702b9f891 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_JSON/6.0.0_1583319045090/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: INNER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/plan.json new file mode 100644 index 000000000000..16e96c083cb4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/spec.json new file mode 100644 index 000000000000..c090eb5d9952 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/spec.json @@ -0,0 +1,119 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045132, + "schemas" : { + "CTAS_INNER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_INNER_JOIN_0.INNER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/topology new file mode 100644 index 000000000000..470702b9f891 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_inner_join_-_PROTOBUF/6.0.0_1583319045132/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: INNER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/plan.json new file mode 100644 index 000000000000..1919c7a41880 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/plan.json @@ -0,0 +1,219 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nWHERE ((T.VALUE > 10) AND (TT.F2 > 5))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableFilterV1", + "properties" : { + "queryContext" : "WhereFilter" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "filterExpression" : "((T_VALUE > 10) AND (TT_F2 > 5))" + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/spec.json new file mode 100644 index 000000000000..7f91f2df5e22 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/spec.json @@ -0,0 +1,117 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045667, + "schemas" : { + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 4 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 90, + "value" : { + "F1" : "b", + "F2" : 10 + }, + "timestamp" : 18000 + }, { + "topic" : "right_topic", + "key" : 90, + "value" : null, + "timestamp" : 19000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : null, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : null, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "F1" : "b", + "F2" : 10 + }, + "timestamp" : 18000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : null, + "timestamp" : 19000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/topology new file mode 100644 index 000000000000..09c852ca2330 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_join_with_where_clause/6.0.0_1583319045667/topology @@ -0,0 +1,51 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> WhereFilter-ApplyPredicate + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: WhereFilter-ApplyPredicate (stores: []) + --> WhereFilter-Filter + <-- KTABLE-MERGE-0000000008 + Processor: WhereFilter-Filter (stores: []) + --> WhereFilter-PostProcess + <-- WhereFilter-ApplyPredicate + Processor: WhereFilter-PostProcess (stores: []) + --> Project + <-- WhereFilter-Filter + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000015 + <-- WhereFilter-PostProcess + Processor: KTABLE-TOSTREAM-0000000015 (stores: []) + --> KSTREAM-SINK-0000000016 + <-- Project + Sink: KSTREAM-SINK-0000000016 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000015 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/plan.json new file mode 100644 index 000000000000..cdcb35fecc7d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/spec.json new file mode 100644 index 000000000000..5e5f5a07fcc4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044900, + "schemas" : { + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/topology new file mode 100644 index 000000000000..12986f11bae5 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_AVRO/6.0.0_1583319044900/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: LEFT_OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/plan.json new file mode 100644 index 000000000000..b34d40b327af --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/spec.json new file mode 100644 index 000000000000..a92ea2a8b87e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319044961, + "schemas" : { + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : null, + "F2" : null + }, + "timestamp" : 17000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/topology new file mode 100644 index 000000000000..12986f11bae5 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_JSON/6.0.0_1583319044961/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: LEFT_OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/plan.json new file mode 100644 index 000000000000..2714f65d0a7a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE LEFT_OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "LEFT_OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "LEFT_OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "LEFT_OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "LEFT_OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "LEFT_OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_LEFT_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/spec.json new file mode 100644 index 000000000000..4dba1c924c21 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045007, + "schemas" : { + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_LEFT_OUTER_JOIN_0.LEFT_OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "LEFT_OUTER_JOIN", + "key" : 90, + "value" : { + "T_ID" : 90, + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/topology new file mode 100644 index 000000000000..12986f11bae5 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_left_join_-_PROTOBUF/6.0.0_1583319045007/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: LEFT_OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/plan.json new file mode 100644 index 000000000000..04f5e2ede4df --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/spec.json new file mode 100644 index 000000000000..9e65db282cb3 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045174, + "schemas" : { + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + } ], + "outputs" : [ { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTER_JOIN", + "key" : 15, + "value" : { + "T_ID" : null, + "NAME" : null, + "VALUE" : null, + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/topology new file mode 100644 index 000000000000..4cb9c9b75745 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_AVRO/6.0.0_1583319045174/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/plan.json new file mode 100644 index 000000000000..62ceb3fefbf7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/spec.json new file mode 100644 index 000000000000..d313ebe678c2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045215, + "schemas" : { + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + } ], + "outputs" : [ { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : null, + "F2" : null + }, + "timestamp" : 0 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : null, + "F2" : null + }, + "timestamp" : 11000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTER_JOIN", + "key" : 15, + "value" : { + "T_ID" : null, + "NAME" : null, + "VALUE" : null, + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/topology new file mode 100644 index 000000000000..4cb9c9b75745 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_JSON/6.0.0_1583319045215/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/plan.json new file mode 100644 index 000000000000..886ed30b99b4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST_TABLE", "TEST" ], + "sink" : "OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT" + }, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT" + }, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ID AS TT_ID" ] + } + }, + "selectExpressions" : [ "T_ID AS T_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTER_JOIN_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/spec.json new file mode 100644 index 000000000000..530f4745d67f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045259, + "schemas" : { + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_OUTER_JOIN_0.OUTER_JOIN" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + } ], + "outputs" : [ { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTER_JOIN", + "key" : 15, + "value" : { + "T_ID" : 0, + "NAME" : "", + "VALUE" : 0, + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "OUTER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/topology new file mode 100644 index 000000000000..4cb9c9b75745 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_outer_join_-_PROTOBUF/6.0.0_1583319045259/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/plan.json new file mode 100644 index 000000000000..c7c65a69a205 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T1', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T2', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T1.NAME NAME1,\n T2.NAME NAME2\nFROM T1 T1\nINNER JOIN T2 T2 ON ((T1.ID = T2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "T1", "T2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T1_NAME", "ROWTIME AS T1_ROWTIME", "ID AS T1_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T2_NAME", "ROWTIME AS T2_ROWTIME", "ID AS T2_ID" ] + } + }, + "selectExpressions" : [ "T1_NAME AS NAME1", "T2_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/spec.json new file mode 100644 index 000000000000..081e498247bf --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/spec.json @@ -0,0 +1,49 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045946, + "schemas" : { + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "T1", + "key" : 0, + "value" : "a", + "timestamp" : 0 + }, { + "topic" : "T2", + "key" : 0, + "value" : "b", + "timestamp" : 10 + }, { + "topic" : "T1", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "T2", + "key" : 0, + "value" : null, + "timestamp" : 30 + }, { + "topic" : "T1", + "key" : 0, + "value" : null, + "timestamp" : 40 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : null, + "timestamp" : 20 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/topology new file mode 100644 index 000000000000..a82739625386 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs/6.0.0_1583319045946/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [T1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [T2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/plan.json new file mode 100644 index 000000000000..69690cc905da --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T1', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T2', VALUE_FORMAT='JSON', WRAP_SINGLE_VALUE=false);", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT WITH (WRAP_SINGLE_VALUE=false) AS SELECT T1.NAME NAME\nFROM T1 T1\nINNER JOIN T2 T2 ON ((T1.ID = T2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "T1", "T2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T1_NAME", "ROWTIME AS T1_ROWTIME", "ID AS T1_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T2_NAME", "ROWTIME AS T2_ROWTIME", "ID AS T2_ID" ] + } + }, + "selectExpressions" : [ "T1_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ "UNWRAP_SINGLE_VALUES" ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/spec.json new file mode 100644 index 000000000000..209eda46253a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/spec.json @@ -0,0 +1,46 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045976, + "schemas" : { + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : "VARCHAR", + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : "VARCHAR", + "CTAS_OUTPUT_0.OUTPUT" : "VARCHAR" + }, + "inputs" : [ { + "topic" : "T1", + "key" : 0, + "value" : "a", + "timestamp" : 0 + }, { + "topic" : "T2", + "key" : 0, + "value" : "b", + "timestamp" : 10 + }, { + "topic" : "T1", + "key" : 0, + "value" : null, + "timestamp" : 20 + }, { + "topic" : "T2", + "key" : 0, + "value" : null, + "timestamp" : 30 + }, { + "topic" : "T1", + "key" : 0, + "value" : null, + "timestamp" : 40 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : "a", + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : null, + "timestamp" : 20 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/topology new file mode 100644 index 000000000000..a82739625386 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_unwrapped_single_field_value_schema_on_inputs_and_output/6.0.0_1583319045976/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [T1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [T2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/plan.json new file mode 100644 index 000000000000..3aa31e4f85b4 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/plan.json @@ -0,0 +1,212 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T1', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T2 (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='T2', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T2", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T1.NAME NAME1,\n T2.NAME NAME2\nFROM T1 T1\nINNER JOIN T2 T2 ON ((T1.ID = T2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME1` STRING, `NAME2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "T1", "T2" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T1_NAME", "ROWTIME AS T1_ROWTIME", "ID AS T1_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "T2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS T2_NAME", "ROWTIME AS T2_ROWTIME", "ID AS T2_ID" ] + } + }, + "selectExpressions" : [ "T1_NAME AS NAME1", "T2_NAME AS NAME2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/spec.json new file mode 100644 index 000000000000..487de8d619cd --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319045918, + "schemas" : { + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "T1", + "key" : 0, + "value" : { + "NAME" : "a" + }, + "timestamp" : 0 + }, { + "topic" : "T2", + "key" : 0, + "value" : { + "NAME" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "T1", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 20 + }, { + "topic" : "T2", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 30 + }, { + "topic" : "T1", + "key" : 0, + "value" : { + "NAME" : null + }, + "timestamp" : 40 + }, { + "topic" : "T1", + "key" : 0, + "value" : null, + "timestamp" : 50 + }, { + "topic" : "T2", + "key" : 0, + "value" : null, + "timestamp" : 60 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : "a", + "NAME2" : "b" + }, + "timestamp" : 10 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : "b" + }, + "timestamp" : 20 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : null + }, + "timestamp" : 30 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME1" : null, + "NAME2" : null + }, + "timestamp" : 40 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : null, + "timestamp" : 50 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/topology b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/topology new file mode 100644 index 000000000000..a82739625386 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/joins_-_table-table_wrapped_single_field_value_schema_on_inputs/6.0.0_1583319045918/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [T1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [T2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- Project + Sink: KSTREAM-SINK-0000000013 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000012 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json new file mode 100644 index 000000000000..c50ef3113a0f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (KEY STRING KEY, ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`KEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n INPUT.KEY KEY2\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`KEY` STRING KEY, `ID` BIGINT, `KEY2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`KEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "KEY AS KEY2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent6357789129346422021", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json new file mode 100644 index 000000000000..5242b6561f65 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/spec.json @@ -0,0 +1,23 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583166643536, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : "a", + "value" : { + "id" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "a", + "value" : { + "ID" : 1, + "KEY2" : "a" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_KEY_key_field_name/6.0.0_1583166643536/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json new file mode 100644 index 000000000000..031a3f1fe847 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (OTHER DOUBLE KEY, ID BIGINT) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`OTHER` DOUBLE KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n INPUT.OTHER KEY\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`OTHER` DOUBLE KEY, `ID` BIGINT, `KEY` DOUBLE", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`OTHER` DOUBLE KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS ID", "OTHER AS KEY" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent6357789129346422021", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json new file mode 100644 index 000000000000..54cc810810a1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/spec.json @@ -0,0 +1,23 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583166643478, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : 3.0, + "value" : { + "id" : 1 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 3.0, + "value" : { + "ID" : 1, + "KEY" : 3.0 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/key-schemas_-_explicit_key_field_named_other_than_ROWKEY/6.0.0_1583166643478/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/plan.json new file mode 100644 index 000000000000..e3a3b8341d9f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nPARTITION BY INPUT.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent335545957785424598", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/spec.json new file mode 100644 index 000000000000..140d6cfe225c --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/spec.json @@ -0,0 +1,49 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583329143853, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 11, + "value" : { + "name" : "a" + }, + "timestamp" : 12345 + }, { + "topic" : "test_topic", + "key" : 10, + "value" : { + "name" : "b" + }, + "timestamp" : 12365 + }, { + "topic" : "test_topic", + "key" : 11, + "value" : { + "name" : "c" + }, + "timestamp" : 12375 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 11, + "value" : { + "NAME" : "a" + } + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "b" + } + }, { + "topic" : "OUTPUT", + "key" : 11, + "value" : { + "NAME" : "c" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/topology b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_only_key_column/6.0.0_1583329143853/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/plan.json new file mode 100644 index 000000000000..5a066f053f28 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='input', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nPARTITION BY INPUT.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/spec.json new file mode 100644 index 000000000000..2c267e53c9c1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319053891, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "input", + "key" : 10, + "value" : { + "NAME" : "bob" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "bob" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/topology b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_partition_by_only_key_column/6.0.0_1583319053891/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/plan.json new file mode 100644 index 000000000000..3cf0adaa97bf --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (`Key` STRING KEY, `Name` STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`Key` STRING KEY, `Name` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n *,\n INPUT.`Name` `Name2`\nFROM INPUT INPUT\nPARTITION BY INPUT.`Key`\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`Key` STRING KEY, `Name` STRING, `Name2` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`Key` STRING KEY, `Name` STRING" + }, + "selectExpressions" : [ "`Name` AS `Name`", "`Name` AS `Name2`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent335545957785424598", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/spec.json new file mode 100644 index 000000000000..853232a5ed9f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/spec.json @@ -0,0 +1,52 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583329143863, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "x", + "value" : { + "Name" : "a" + }, + "timestamp" : 12345 + }, { + "topic" : "test_topic", + "key" : "y", + "value" : { + "Name" : "b" + }, + "timestamp" : 12365 + }, { + "topic" : "test_topic", + "key" : "x", + "value" : { + "Name" : "c" + }, + "timestamp" : 12375 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "Name" : "a", + "Name2" : "a" + } + }, { + "topic" : "OUTPUT", + "key" : "y", + "value" : { + "Name" : "b", + "Name2" : "b" + } + }, { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "Name" : "c", + "Name2" : "c" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/topology b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/partition-by_-_should_handled_quoted_key_and_value/6.0.0_1583329143863/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/plan.json new file mode 100644 index 000000000000..35437ae33108 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID_COPY,\n INPUT.ROWTIME ROWTIME_COPY,\n INPUT.NAME NAME\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `ID_COPY` INTEGER, `ROWTIME_COPY` BIGINT, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "ID AS ID_COPY", "ROWTIME AS ROWTIME_COPY", "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/spec.json new file mode 100644 index 000000000000..b0c96b410c73 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/spec.json @@ -0,0 +1,26 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319054694, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "a" + }, + "timestamp" : 1234 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "ID_COPY" : 8, + "ROWTIME_COPY" : 1234, + "NAME" : "a" + }, + "timestamp" : 1234 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/topology b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_explicit/6.0.0_1583319054694/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/plan.json new file mode 100644 index 000000000000..e53634e3529f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/spec.json new file mode 100644 index 000000000000..afaf3fc8c3d0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319054687, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "a" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "NAME" : "a" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/topology b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_all_columns_-_star/6.0.0_1583319054687/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/plan.json new file mode 100644 index 000000000000..e95819af3f4e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT INPUT.ID ID_COPY\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `ID_COPY` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "ID AS ID_COPY" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/spec.json new file mode 100644 index 000000000000..9ab016a28494 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319054671, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "a" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "ID_COPY" : 8 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/topology b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_key_column/6.0.0_1583319054671/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/plan.json new file mode 100644 index 000000000000..4a31c6e64fb7 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/plan.json @@ -0,0 +1,147 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT INPUT.NAME NAME\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/spec.json new file mode 100644 index 000000000000..11413fc01cdb --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319054680, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "a" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "NAME" : "a" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/topology b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/select_-_value_column/6.0.0_1583319054680/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/plan.json new file mode 100644 index 000000000000..18b697858dee --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/plan.json @@ -0,0 +1,154 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT TEST_UDTF(TEST.ID) KSQL_COL_0\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `KSQL_COL_0` INTEGER", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamFlatMapV1", + "properties" : { + "queryContext" : "FlatMap" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "tableFunctions" : [ "TEST_UDTF(ID)" ] + }, + "selectExpressions" : [ "KSQL_SYNTH_0 AS KSQL_COL_0" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/spec.json new file mode 100644 index 000000000000..ccb693eb0faa --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/spec.json @@ -0,0 +1,22 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319056601, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 10, + "value" : { + "NAME" : "bob" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "KSQL_COL_0" : 10 + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/topology b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/topology new file mode 100644 index 000000000000..a266798d8e3b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/table-functions_-_test_udtf_-_key_column/6.0.0_1583319056601/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> FlatMap + <-- KSTREAM-SOURCE-0000000000 + Processor: FlatMap (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- FlatMap + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/plan.json new file mode 100644 index 000000000000..dcb37adfd169 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/plan.json @@ -0,0 +1,154 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nWHERE (INPUT.ID < 10)\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamFilterV1", + "properties" : { + "queryContext" : "WhereFilter" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "filterExpression" : "(ID < 10)" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/spec.json new file mode 100644 index 000000000000..83d57795a2ac --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/spec.json @@ -0,0 +1,46 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319057678, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "a" + } + }, { + "topic" : "test_topic", + "key" : 9, + "value" : { + "name" : "a" + } + }, { + "topic" : "test_topic", + "key" : 10, + "value" : { + "name" : "b" + } + }, { + "topic" : "test_topic", + "key" : 11, + "value" : { + "name" : "c" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "NAME" : "a" + } + }, { + "topic" : "OUTPUT", + "key" : 9, + "value" : { + "NAME" : "a" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/topology b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/topology new file mode 100644 index 000000000000..7b6119f7a4ae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_key_column/6.0.0_1583319057678/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> WhereFilter + <-- KSTREAM-SOURCE-0000000000 + Processor: WhereFilter (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- WhereFilter + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/plan.json new file mode 100644 index 000000000000..e03e44d7cab9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/plan.json @@ -0,0 +1,154 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID INTEGER KEY, NAME STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nWHERE (NOT (INPUT.NAME LIKE '%not%'))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` INTEGER KEY, `NAME` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamFilterV1", + "properties" : { + "queryContext" : "WhereFilter" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` INTEGER KEY, `NAME` STRING" + }, + "filterExpression" : "(NOT (NAME LIKE '%not%'))" + }, + "selectExpressions" : [ "NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/var/folders/2d/3pt97ylj3zngd51bwl91bl3r0000gp/T/confluent3333823077133685458", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.any.key.name.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/spec.json new file mode 100644 index 000000000000..1883c12f19db --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/spec.json @@ -0,0 +1,46 @@ +{ + "version" : "6.0.0", + "timestamp" : 1583319057687, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : 8, + "value" : { + "name" : "this one" + } + }, { + "topic" : "test_topic", + "key" : 9, + "value" : { + "name" : "not this one" + } + }, { + "topic" : "test_topic", + "key" : 10, + "value" : { + "name" : "and this one" + } + }, { + "topic" : "test_topic", + "key" : 11, + "value" : { + "name" : "but not this one" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 8, + "value" : { + "NAME" : "this one" + } + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "and this one" + } + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/topology b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/topology new file mode 100644 index 000000000000..7b6119f7a4ae --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/where_-_on_value_column/6.0.0_1583319057687/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> WhereFilter + <-- KSTREAM-SOURCE-0000000000 + Processor: WhereFilter (stores: []) + --> Project + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000004 + <-- WhereFilter + Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) + <-- Project +