diff --git a/docs/developer-guide/ksqldb-reference/scalar-functions.md b/docs/developer-guide/ksqldb-reference/scalar-functions.md index fa91977f01ae..cd62b8fa229c 100644 --- a/docs/developer-guide/ksqldb-reference/scalar-functions.md +++ b/docs/developer-guide/ksqldb-reference/scalar-functions.md @@ -380,6 +380,17 @@ SLICE(col1, from, to) Slices a list based on the supplied indices. The indices start at 1 and include both endpoints. +### `ARRAY_JOIN` + +```sql +ARRAY_JOIN(col1, delimiter) +``` + +Creates a flat string representation of all the elements contained in the given array. +The elements in the resulting string are separated by the chosen `delimiter`, +which is an optional parameter that falls back to a comma `,`. The current implementation only +allows for array elements of primitive ksqlDB types. + ## Strings ### `CHR` diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java new file mode 100644 index 000000000000..ebabda5e58fb --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java @@ -0,0 +1,84 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udf.array; + +import com.google.common.collect.ImmutableSet; +import io.confluent.ksql.function.KsqlFunctionException; +import io.confluent.ksql.function.udf.Udf; +import io.confluent.ksql.function.udf.UdfDescription; +import io.confluent.ksql.function.udf.UdfParameter; +import io.confluent.ksql.util.KsqlConstants; +import java.math.BigDecimal; +import java.util.List; +import java.util.Set; +import java.util.StringJoiner; + +@SuppressWarnings("MethodMayBeStatic") // UDF methods can not be static. +@UdfDescription( + name = "ARRAY_JOIN", + description = "joins the array elements into a flat string representation", + author = KsqlConstants.CONFLUENT_AUTHOR +) +public class ArrayJoin { + + private static final String DEFAULT_DELIMITER = ","; + private static final Set KSQL_PRIMITIVES = ImmutableSet.of( + Boolean.class,Integer.class,Long.class,Double.class,BigDecimal.class,String.class + ); + + @Udf + public String join( + @UdfParameter(description = "the array to join using the default delimiter '" + + DEFAULT_DELIMITER + "'") final List array + ) { + return join(array, DEFAULT_DELIMITER); + } + + @Udf + public String join( + @UdfParameter(description = "the array to join using the specified delimiter") + final List array, + @UdfParameter(description = "the string to be used as element delimiter") + final String delimiter + ) { + + if (array == null) { + return null; + } + + final StringJoiner sj = new StringJoiner(delimiter == null ? "" : delimiter); + array.forEach(e -> processElement(e, sj)); + return sj.toString(); + + } + + @SuppressWarnings("unchecked") + private static void processElement(final T element, final StringJoiner joiner) { + + if (element == null || KSQL_PRIMITIVES.contains(element.getClass())) { + handlePrimitiveType(element, joiner); + } else { + throw new KsqlFunctionException("error: hit element of type " + + element.getClass().getTypeName() + " which is currently not supported"); + } + + } + + private static void handlePrimitiveType(final Object element, final StringJoiner joiner) { + joiner.add(element != null ? element.toString() : null); + } + +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java new file mode 100644 index 000000000000..d15e841b55cb --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java @@ -0,0 +1,131 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udf.array; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThrows; + +import io.confluent.ksql.function.KsqlFunctionException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import org.junit.Test; + +public class ArrayJoinTest { + + private static final String CUSTOM_DELIMITER = "|"; + + private final ArrayJoin arrayJoinUDF = new ArrayJoin(); + + @Test + public void shouldReturnNullForNullInput() { + assertThat(arrayJoinUDF.join(null), nullValue()); + assertThat(arrayJoinUDF.join(null,CUSTOM_DELIMITER), nullValue()); + } + + @Test + public void shouldReturnEmptyStringForEmptyArrays() { + assertThat(arrayJoinUDF.join(Collections.emptyList()).isEmpty(),is(true)); + assertThat(arrayJoinUDF.join(Collections.emptyList(),CUSTOM_DELIMITER).isEmpty(),is(true)); + } + + @Test + public void shouldReturnCorrectStringForFlatArraysWithPrimitiveTypes() { + + assertThat(arrayJoinUDF.join(Arrays.asList(true, null, false),""), + is("truenullfalse") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(true, null, false)), + is("true,null,false") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(true,null,false),CUSTOM_DELIMITER), + is("true"+CUSTOM_DELIMITER+"null"+CUSTOM_DELIMITER+"false") + ); + + assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0),null), is("123-420")); + assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0)), is("1,23,-42,0")); + assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0),CUSTOM_DELIMITER), + is("1"+CUSTOM_DELIMITER+"23"+CUSTOM_DELIMITER+"-42"+CUSTOM_DELIMITER+"0") + ); + + assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L),""), + is("-42949672978589934592") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L)), + is("-4294967297,8589934592") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L), CUSTOM_DELIMITER), + is("-4294967297"+CUSTOM_DELIMITER+"8589934592") + ); + + assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0),null), + is("1.23-23.420.0") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0)), + is("1.23,-23.42,0.0") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0),CUSTOM_DELIMITER), + is("1.23"+CUSTOM_DELIMITER+"-23.42"+CUSTOM_DELIMITER+"0.0") + ); + + assertThat(arrayJoinUDF.join( + Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65")),null + ), + is("123.45987.65") + ); + assertThat(arrayJoinUDF.join(Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65"))), + is("123.45,987.65") + ); + assertThat(arrayJoinUDF.join( + Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65")),CUSTOM_DELIMITER), + is("123.45"+CUSTOM_DELIMITER+"987.65") + ); + + assertThat(arrayJoinUDF.join(Arrays.asList("Hello","From","","Ksqldb","Udf"),""), + is("HelloFromKsqldbUdf") + ); + assertThat(arrayJoinUDF.join(Arrays.asList("Hello","From","","Ksqldb","Udf")), + is("Hello,From,,Ksqldb,Udf") + ); + assertThat( + arrayJoinUDF.join(Arrays.asList("hello","from","","ksqldb","udf",null),CUSTOM_DELIMITER), + is("hello"+CUSTOM_DELIMITER+"from"+CUSTOM_DELIMITER+CUSTOM_DELIMITER + +"ksqldb"+CUSTOM_DELIMITER+"udf"+CUSTOM_DELIMITER+"null") + ); + + } + + @Test + public void shouldThrowExceptionForExamplesOfUnsupportedElementTypes() { + assertThrows(KsqlFunctionException.class, + () -> arrayJoinUDF.join(Arrays.asList('a','b'))); + assertThrows(KsqlFunctionException.class, + () -> arrayJoinUDF.join(Arrays.asList(BigInteger.ONE,BigInteger.ZERO))); + assertThrows(KsqlFunctionException.class, + () -> arrayJoinUDF.join(Arrays.asList(-23.0f,42.42f,0.0f))); + assertThrows(KsqlFunctionException.class, + () -> arrayJoinUDF.join(Arrays.asList( + new HashSet<>(Arrays.asList("foo", "blah")), + new HashSet<>(Arrays.asList("ksqlDB", "UDF")) + )) + ); + } + +} diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json new file mode 100644 index 000000000000..b5535fca092f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n ARRAY_JOIN(TEST.BOOLEANARRAY, '|') KSQL_COL_0,\n ARRAY_JOIN(TEST.INTARRAY, '?') KSQL_COL_1,\n ARRAY_JOIN(TEST.BIGINTARRAY, ';') KSQL_COL_2,\n ARRAY_JOIN(TEST.DOUBLEARRAY, ' ') KSQL_COL_3,\n ARRAY_JOIN(TEST.DECIMALARRAY, '#') KSQL_COL_4,\n ARRAY_JOIN(TEST.STRINGARRAY, '_') KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY, '|') AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY, '?') AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY, ';') AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY, ' ') AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY, '#') AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY, '_') AS KSQL_COL_5" ] + }, + "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, + "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.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.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "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.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", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json new file mode 100644 index 000000000000..12a2c5dfee3b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json @@ -0,0 +1,73 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591955240274, + "path" : "query-validation-tests/arrayjoin.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "join flat arrays with primitive ksqldb types and custom delimiters", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "BOOLEANARRAY" : [ true, null, false ], + "INTARRAY" : [ 1, 23, -42, 0 ], + "BIGINTARRAY" : [ -4294967297, 8589934592 ], + "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ], + "DECIMALARRAY" : [ 123.45, 987.65 ], + "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "KSQL_COL_0" : "true|null|false", + "KSQL_COL_1" : "1?23?-42?0", + "KSQL_COL_2" : "-4294967297;8589934592", + "KSQL_COL_3" : "1.23 -23.42 0.0", + "KSQL_COL_4" : "123.45#987.65", + "KSQL_COL_5" : "Hello_From__Ksqldb_Udf" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,'|'), ARRAY_JOIN(INTARRAY,'?'), ARRAY_JOIN(BIGINTARRAY,';'), ARRAY_JOIN(DOUBLEARRAY,' '), ARRAY_JOIN(DECIMALARRAY,'#'), ARRAY_JOIN(STRINGARRAY,'_') FROM TEST;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json new file mode 100644 index 000000000000..75e075eb8ae4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n ARRAY_JOIN(TEST.BOOLEANARRAY) KSQL_COL_0,\n ARRAY_JOIN(TEST.INTARRAY) KSQL_COL_1,\n ARRAY_JOIN(TEST.BIGINTARRAY) KSQL_COL_2,\n ARRAY_JOIN(TEST.DOUBLEARRAY) KSQL_COL_3,\n ARRAY_JOIN(TEST.DECIMALARRAY) KSQL_COL_4,\n ARRAY_JOIN(TEST.STRINGARRAY) KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY) AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY) AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY) AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY) AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY) AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY) AS KSQL_COL_5" ] + }, + "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, + "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.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.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "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.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", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json new file mode 100644 index 000000000000..7fc811f9adb6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json @@ -0,0 +1,73 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591953360045, + "path" : "query-validation-tests/arrayjoin.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "join flat arrays with primitive ksqldb types and default delimiter", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "BOOLEANARRAY" : [ true, null, false ], + "INTARRAY" : [ 1, 23, -42, 0 ], + "BIGINTARRAY" : [ -4294967297, 8589934592 ], + "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ], + "DECIMALARRAY" : [ 123.45, 987.65 ], + "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "KSQL_COL_0" : "true,null,false", + "KSQL_COL_1" : "1,23,-42,0", + "KSQL_COL_2" : "-4294967297,8589934592", + "KSQL_COL_3" : "1.23,-23.42,0.0", + "KSQL_COL_4" : "123.45,987.65", + "KSQL_COL_5" : "Hello,From,,Ksqldb,Udf" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY), ARRAY_JOIN(INTARRAY), ARRAY_JOIN(BIGINTARRAY), ARRAY_JOIN(DOUBLEARRAY), ARRAY_JOIN(DECIMALARRAY), ARRAY_JOIN(STRINGARRAY) FROM TEST;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json new file mode 100644 index 000000000000..64eb4ec14fbb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n ARRAY_JOIN(TEST.BOOLEANARRAY, '') KSQL_COL_0,\n ARRAY_JOIN(TEST.INTARRAY, null) KSQL_COL_1,\n ARRAY_JOIN(TEST.BIGINTARRAY, '') KSQL_COL_2,\n ARRAY_JOIN(TEST.DOUBLEARRAY, null) KSQL_COL_3,\n ARRAY_JOIN(TEST.DECIMALARRAY, '') KSQL_COL_4,\n ARRAY_JOIN(TEST.STRINGARRAY, null) KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY, `INTARRAY` ARRAY, `BIGINTARRAY` ARRAY, `DOUBLEARRAY` ARRAY, `DECIMALARRAY` ARRAY, `STRINGARRAY` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY, '') AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY, null) AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY, '') AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY, null) AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY, '') AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY, null) AS KSQL_COL_5" ] + }, + "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, + "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.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.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "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.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", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json new file mode 100644 index 000000000000..7d30338528ad --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json @@ -0,0 +1,73 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591955240399, + "path" : "query-validation-tests/arrayjoin.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY> NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "join flat arrays with primitive ksqldb types and empty or null delimiter", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "BOOLEANARRAY" : [ true, null, false ], + "INTARRAY" : [ 1, 23, -42, 0 ], + "BIGINTARRAY" : [ -4294967297, 8589934592 ], + "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ], + "DECIMALARRAY" : [ 123.45, 987.65 ], + "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "KSQL_COL_0" : "truenullfalse", + "KSQL_COL_1" : "123-420", + "KSQL_COL_2" : "-42949672978589934592", + "KSQL_COL_3" : "1.23-23.420.0", + "KSQL_COL_4" : "123.45987.65", + "KSQL_COL_5" : "HelloFromKsqldbUdf" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,''), ARRAY_JOIN(INTARRAY,null), ARRAY_JOIN(BIGINTARRAY,''), ARRAY_JOIN(DOUBLEARRAY,null), ARRAY_JOIN(DECIMALARRAY,''), ARRAY_JOIN(STRINGARRAY,null) FROM TEST;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json new file mode 100644 index 000000000000..ad0a7a5801a6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json @@ -0,0 +1,109 @@ +{ + "comments": [ + "Tests covering the use of the ARRAY_JOIN function." + ], + "tests": [ + { + "name": "join flat arrays with primitive ksqldb types and default delimiter", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY), ARRAY_JOIN(INTARRAY), ARRAY_JOIN(BIGINTARRAY), ARRAY_JOIN(DOUBLEARRAY), ARRAY_JOIN(DECIMALARRAY), ARRAY_JOIN(STRINGARRAY) FROM TEST;" + ], + "inputs": [ + { + "topic": "test_topic", "key": "1", + "value": { + "BOOLEANARRAY": [true, null, false], + "INTARRAY": [1, 23, -42, 0], + "BIGINTARRAY": [-4294967297, 8589934592], + "DOUBLEARRAY": [1.23, -23.42, 0.0], + "DECIMALARRAY": [123.45, 987.65], + "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"] + } + } + ], + "outputs": [ + { + "topic": "OUTPUT", + "key": "1", + "value": { + "KSQL_COL_0": "true,null,false", + "KSQL_COL_1": "1,23,-42,0", + "KSQL_COL_2": "-4294967297,8589934592", + "KSQL_COL_3": "1.23,-23.42,0.0", + "KSQL_COL_4": "123.45,987.65", + "KSQL_COL_5": "Hello,From,,Ksqldb,Udf" + } + } + ] + }, + { + "name": "join flat arrays with primitive ksqldb types and custom delimiters", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,'|'), ARRAY_JOIN(INTARRAY,'?'), ARRAY_JOIN(BIGINTARRAY,';'), ARRAY_JOIN(DOUBLEARRAY,' '), ARRAY_JOIN(DECIMALARRAY,'#'), ARRAY_JOIN(STRINGARRAY,'_') FROM TEST;" + ], + "inputs": [ + { + "topic": "test_topic", "key": "1", + "value": { + "BOOLEANARRAY": [true, null, false], + "INTARRAY": [1, 23, -42, 0], + "BIGINTARRAY": [-4294967297, 8589934592], + "DOUBLEARRAY": [1.23, -23.42, 0.0], + "DECIMALARRAY": [123.45, 987.65], + "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"] + } + } + ], + "outputs": [ + { + "topic": "OUTPUT", + "key": "1", + "value": { + "KSQL_COL_0": "true|null|false", + "KSQL_COL_1": "1?23?-42?0", + "KSQL_COL_2": "-4294967297;8589934592", + "KSQL_COL_3": "1.23 -23.42 0.0", + "KSQL_COL_4": "123.45#987.65", + "KSQL_COL_5": "Hello_From__Ksqldb_Udf" + } + } + ] + }, + { + "name": "join flat arrays with primitive ksqldb types and empty or null delimiter", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY, INTARRAY ARRAY, BIGINTARRAY ARRAY, DOUBLEARRAY ARRAY, DECIMALARRAY ARRAY, STRINGARRAY ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,''), ARRAY_JOIN(INTARRAY,null), ARRAY_JOIN(BIGINTARRAY,''), ARRAY_JOIN(DOUBLEARRAY,null), ARRAY_JOIN(DECIMALARRAY,''), ARRAY_JOIN(STRINGARRAY,null) FROM TEST;" + ], + "inputs": [ + { + "topic": "test_topic", "key": "1", + "value": { + "BOOLEANARRAY": [true, null, false], + "INTARRAY": [1, 23, -42, 0], + "BIGINTARRAY": [-4294967297, 8589934592], + "DOUBLEARRAY": [1.23, -23.42, 0.0], + "DECIMALARRAY": [123.45, 987.65], + "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"] + } + } + ], + "outputs": [ + { + "topic": "OUTPUT", + "key": "1", + "value": { + "KSQL_COL_0": "truenullfalse", + "KSQL_COL_1": "123-420", + "KSQL_COL_2": "-42949672978589934592", + "KSQL_COL_3": "1.23-23.420.0", + "KSQL_COL_4": "123.45987.65", + "KSQL_COL_5": "HelloFromKsqldbUdf" + } + } + ] + } + ] +} \ No newline at end of file