Skip to content

Commit

Permalink
fix: allow expressions in flat map (#6163)
Browse files Browse the repository at this point in the history
  • Loading branch information
vcrfxia committed Sep 9, 2020
1 parent 96bb12a commit 52a897b
Show file tree
Hide file tree
Showing 8 changed files with 616 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@
public class FlatMapNode extends SingleSourcePlanNode {

private final ImmutableList<FunctionCall> tableFunctions;
private final ImmutableMap<Integer, UnqualifiedColumnReferenceExp> columnMappings;
private final ImmutableMap<Integer, Expression> columnMappings;
private final LogicalSchema schema;

public FlatMapNode(
Expand Down Expand Up @@ -89,14 +89,14 @@ public SchemaKStream<?> buildStream(final KsqlQueryBuilder builder) {
);
}

private static ImmutableMap<Integer, UnqualifiedColumnReferenceExp> buildColumnMappings(
private static ImmutableMap<Integer, Expression> buildColumnMappings(
final FunctionRegistry functionRegistry,
final ImmutableAnalysis analysis
) {
final TableFunctionExpressionRewriter tableFunctionExpressionRewriter =
new TableFunctionExpressionRewriter(functionRegistry);

final ImmutableMap.Builder<Integer, UnqualifiedColumnReferenceExp> builder = ImmutableMap
final ImmutableMap.Builder<Integer, Expression> builder = ImmutableMap
.builder();

for (int idx = 0; idx < analysis.getSelectItems().size(); idx++) {
Expand All @@ -110,7 +110,7 @@ private static ImmutableMap<Integer, UnqualifiedColumnReferenceExp> buildColumnM
tableFunctionExpressionRewriter::process, singleColumn.getExpression());

if (!rewritten.equals(singleColumn.getExpression())) {
builder.put(idx, (UnqualifiedColumnReferenceExp) rewritten);
builder.put(idx, rewritten);
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
{
"plan" : [ {
"@type" : "ksqlPlanV1",
"statementText" : "CREATE STREAM TEST (ID STRING KEY, MY_ARR ARRAY<STRUCT<F1 BIGINT>>) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
"ddlCommand" : {
"@type" : "createStreamV1",
"sourceName" : "TEST",
"schema" : "`ID` STRING KEY, `MY_ARR` ARRAY<STRUCT<`F1` BIGINT>>",
"topicName" : "test_topic",
"formats" : {
"keyFormat" : {
"format" : "KAFKA"
},
"valueFormat" : {
"format" : "JSON"
}
},
"orReplace" : false
}
}, {
"@type" : "ksqlPlanV1",
"statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n EXPLODE(TEST.MY_ARR) VAL\nFROM TEST TEST\nEMIT CHANGES",
"ddlCommand" : {
"@type" : "createStreamV1",
"sourceName" : "OUTPUT",
"schema" : "`ID` STRING KEY, `VAL` STRUCT<`F1` BIGINT>",
"topicName" : "OUTPUT",
"formats" : {
"keyFormat" : {
"format" : "KAFKA"
},
"valueFormat" : {
"format" : "JSON"
}
},
"orReplace" : false
},
"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"
},
"valueFormat" : {
"format" : "JSON"
}
},
"sourceSchema" : "`ID` STRING KEY, `MY_ARR` ARRAY<STRUCT<`F1` BIGINT>>"
},
"tableFunctions" : [ "EXPLODE(MY_ARR)" ]
},
"keyColumnNames" : [ "ID" ],
"selectExpressions" : [ "KSQL_SYNTH_0 AS VAL" ]
},
"formats" : {
"keyFormat" : {
"format" : "KAFKA"
},
"valueFormat" : {
"format" : "JSON"
}
},
"topicName" : "OUTPUT"
},
"queryId" : "CSAS_OUTPUT_0"
}
} ],
"configs" : {
"ksql.extension.dir" : "ext",
"ksql.streams.cache.max.bytes.buffering" : "0",
"ksql.security.extension.class" : null,
"metric.reporters" : "",
"ksql.transient.prefix" : "transient_",
"ksql.query.status.running.threshold.seconds" : "300",
"ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
"ksql.output.topic.name.prefix" : "",
"ksql.query.pull.enable.standby.reads" : "false",
"ksql.streams.max.task.idle.ms" : "0",
"ksql.query.error.max.queue.size" : "10",
"ksql.internal.topic.min.insync.replicas" : "1",
"ksql.streams.shutdown.timeout.ms" : "300000",
"ksql.internal.topic.replicas" : "1",
"ksql.insert.into.values.enabled" : "true",
"ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
"ksql.query.pull.max.qps" : "2147483647",
"ksql.access.validator.enable" : "auto",
"ksql.streams.bootstrap.servers" : "localhost:0",
"ksql.query.pull.metrics.enabled" : "false",
"ksql.create.or.replace.enabled" : "true",
"ksql.metrics.extension" : null,
"ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
"ksql.cast.strings.preserve.nulls" : "true",
"ksql.authorization.cache.max.entries" : "10000",
"ksql.pull.queries.enable" : "true",
"ksql.suppress.enabled" : "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.query.persistent.active.limit" : "2147483647",
"ksql.persistence.wrap.single.values" : null,
"ksql.authorization.cache.expiry.time.secs" : "30",
"ksql.query.retry.backoff.initial.ms" : "15000",
"ksql.schema.registry.url" : "",
"ksql.properties.overrides.denylist" : "",
"ksql.streams.auto.offset.reset" : "earliest",
"ksql.connect.url" : "http://localhost:8083",
"ksql.service.id" : "some.ksql.service.id",
"ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
"ksql.enable.metastore.backup" : "false",
"ksql.streams.commit.interval.ms" : "2000",
"ksql.streams.auto.commit.interval.ms" : "0",
"ksql.streams.topology.optimization" : "all",
"ksql.query.retry.backoff.max.ms" : "900000",
"ksql.streams.num.stream.threads" : "4",
"ksql.timestamp.throw.on.invalid" : "false",
"ksql.metrics.tags.custom" : "",
"ksql.udfs.enabled" : "true",
"ksql.udf.enable.security.manager" : "true",
"ksql.connect.worker.config" : "",
"ksql.udf.collect.metrics" : "false",
"ksql.persistent.prefix" : "query_",
"ksql.metastore.backup.location" : "",
"ksql.error.classifier.regex" : "",
"ksql.suppress.buffer.size.bytes" : "-1"
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
{
"version" : "6.1.0",
"timestamp" : 1599586293527,
"path" : "query-validation-tests/explode.json",
"schemas" : {
"CSAS_OUTPUT_0.KsqlTopic.Source" : {
"schema" : "`ID` STRING KEY, `MY_ARR` ARRAY<STRUCT<`F1` BIGINT>>",
"serdeOptions" : [ ]
},
"CSAS_OUTPUT_0.OUTPUT" : {
"schema" : "`ID` STRING KEY, `VAL` STRUCT<`F1` BIGINT>",
"serdeOptions" : [ ]
}
},
"testCase" : {
"name" : "explode array of struct",
"inputs" : [ {
"topic" : "test_topic",
"key" : "0",
"value" : {
"MY_ARR" : [ {
"F1" : 1
}, {
"F1" : 2
} ]
}
}, {
"topic" : "test_topic",
"key" : "1",
"value" : {
"MY_ARR" : [ {
"F1" : 3
} ]
}
}, {
"topic" : "test_topic",
"key" : "2",
"value" : {
"MY_ARR" : [ ]
}
}, {
"topic" : "test_topic",
"key" : "3",
"value" : {
"MY_ARR" : null
}
} ],
"outputs" : [ {
"topic" : "OUTPUT",
"key" : "0",
"value" : {
"VAL" : {
"F1" : 1
}
}
}, {
"topic" : "OUTPUT",
"key" : "0",
"value" : {
"VAL" : {
"F1" : 2
}
}
}, {
"topic" : "OUTPUT",
"key" : "1",
"value" : {
"VAL" : {
"F1" : 3
}
}
} ],
"topics" : [ {
"name" : "OUTPUT",
"replicas" : 1,
"numPartitions" : 4
}, {
"name" : "test_topic",
"replicas" : 1,
"numPartitions" : 4
} ],
"statements" : [ "CREATE STREAM TEST (ID STRING KEY, MY_ARR ARRAY<STRUCT<F1 BIGINT>>) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, EXPLODE(MY_ARR) VAL FROM TEST;" ],
"post" : {
"sources" : [ {
"name" : "TEST",
"type" : "STREAM",
"schema" : "`ID` STRING KEY, `MY_ARR` ARRAY<STRUCT<`F1` BIGINT>>",
"keyFormat" : {
"format" : "KAFKA"
},
"serdeOptions" : [ ]
}, {
"name" : "OUTPUT",
"type" : "STREAM",
"schema" : "`ID` STRING KEY, `VAL` STRUCT<`F1` BIGINT>",
"keyFormat" : {
"format" : "KAFKA"
},
"serdeOptions" : [ ]
} ],
"topics" : {
"topics" : [ {
"name" : "test_topic",
"keyFormat" : {
"formatInfo" : {
"format" : "KAFKA"
}
},
"valueFormat" : {
"format" : "JSON"
},
"partitions" : 4
}, {
"name" : "OUTPUT",
"keyFormat" : {
"formatInfo" : {
"format" : "KAFKA"
}
},
"valueFormat" : {
"format" : "JSON"
},
"partitions" : 4
} ]
}
}
}
}
Original file line number Diff line number Diff line change
@@ -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

Loading

0 comments on commit 52a897b

Please sign in to comment.