From 563ff9b91a9095045c1597a7008359439a8a217c Mon Sep 17 00:00:00 2001 From: Steven Zhang <35498506+stevenpyzhang@users.noreply.github.com> Date: Wed, 24 Feb 2021 11:15:13 -0800 Subject: [PATCH] feat: introduce transform and reduce invocation function for lambdas (#6994) --- .../function/udf/array/ArrayTransform.java | 55 ----- .../ksql/function/udf/lambda/Reduce.java | 96 ++++++++ .../Transform.java} | 47 +++- .../ksql/function/udf/map/ReduceMap.java | 60 ----- .../ksql/function/UdfLoaderTest.java | 10 +- .../udf/array/ArrayTransformTest.java | 83 ------- .../ksql/function/udf/lambda/ReduceTest.java | 138 +++++++++++ .../function/udf/lambda/TransformTest.java | 152 ++++++++++++ .../function/udf/map/MapTransformTest.java | 81 ------ .../ksql/function/udf/map/ReduceMapTest.java | 77 ------ .../execution/util/ExpressionTypeManager.java | 2 +- .../7.0.0_1614136896616/plan.json | 145 +++++++++++ .../7.0.0_1614136896616/spec.json | 120 +++++++++ .../7.0.0_1614136896616/topology | 13 + .../7.0.0_1614136896674/plan.json | 145 +++++++++++ .../7.0.0_1614136896674/spec.json | 110 +++++++++ .../7.0.0_1614136896674/topology | 13 + .../7.0.0_1614069675859/plan.json | 145 +++++++++++ .../7.0.0_1614069675859/spec.json | 128 ++++++++++ .../7.0.0_1614069675859/topology | 13 + .../7.0.0_1614069675904/plan.json | 145 +++++++++++ .../7.0.0_1614069675904/spec.json | 113 +++++++++ .../7.0.0_1614069675904/topology | 13 + .../7.0.0_1614069966647/plan.json | 145 +++++++++++ .../7.0.0_1614069966647/spec.json | 120 +++++++++ .../7.0.0_1614069966647/topology | 13 + .../7.0.0_1614070842316/plan.json | 145 +++++++++++ .../7.0.0_1614070842316/spec.json | 180 ++++++++++++++ .../7.0.0_1614070842316/topology | 13 + .../7.0.0_1614070716263/plan.json | 145 +++++++++++ .../7.0.0_1614070716263/spec.json | 162 ++++++++++++ .../7.0.0_1614070716263/topology | 13 + .../7.0.0_1614070842246/plan.json | 145 +++++++++++ .../7.0.0_1614070842246/spec.json | 174 +++++++++++++ .../7.0.0_1614070842246/topology | 13 + .../7.0.0_1614070842293/plan.json | 145 +++++++++++ .../7.0.0_1614070842293/spec.json | 201 +++++++++++++++ .../7.0.0_1614070842293/topology | 13 + .../7.0.0_1614073059361/plan.json | 145 +++++++++++ .../7.0.0_1614073059361/spec.json | 230 ++++++++++++++++++ .../7.0.0_1614073059361/topology | 13 + .../7.0.0_1614072121318/plan.json | 145 +++++++++++ .../7.0.0_1614072121318/spec.json | 217 +++++++++++++++++ .../7.0.0_1614072121318/topology | 13 + .../7.0.0_1614072565004/plan.json | 145 +++++++++++ .../7.0.0_1614072565004/spec.json | 221 +++++++++++++++++ .../7.0.0_1614072565004/topology | 13 + .../query-validation-tests/reduce-array.json | 39 +++ .../query-validation-tests/reduce-map.json | 39 +++ .../transform-array.json | 98 ++++++++ .../query-validation-tests/transform-map.json | 56 +++++ .../ksql/function/FunctionCategory.java | 1 + 52 files changed, 4531 insertions(+), 375 deletions(-) delete mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayTransform.java create mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Reduce.java rename ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/{map/MapTransform.java => lambda/Transform.java} (52%) delete mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/ReduceMap.java delete mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayTransformTest.java create mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java create mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/TransformTest.java delete mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapTransformTest.java delete mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/ReduceMapTest.java create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/topology create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/plan.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/spec.json create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/topology create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-array.json create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-map.json create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-array.json create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-map.json diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayTransform.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayTransform.java deleted file mode 100644 index 26aaa1537fd1..000000000000 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayTransform.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright 2021 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 io.confluent.ksql.function.FunctionCategory; -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.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * Transform an array with a function - */ -@SuppressWarnings("MethodMayBeStatic") // UDF methods can not be static. -@UdfDescription( - name = "array_transform", - category = FunctionCategory.ARRAY, - description = "Apply a function to each element in an array. " - + "The transformed array is returned.", - author = KsqlConstants.CONFLUENT_AUTHOR -) -public class ArrayTransform { - - @Udf - public List arrayTransform( - @UdfParameter(description = "The array") final List array, - @UdfParameter(description = "The lambda function") final Function function - ) { - if (array == null) { - return null; - } - return array.stream().map(item -> { - if (item == null) { - return null; - } - return function.apply(item); - }).collect(Collectors.toList()); - } -} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Reduce.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Reduce.java new file mode 100644 index 000000000000..57cc2accc36a --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Reduce.java @@ -0,0 +1,96 @@ +/* + * Copyright 2021 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.lambda; + +import io.confluent.ksql.execution.codegen.helpers.TriFunction; +import io.confluent.ksql.function.FunctionCategory; +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.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.BiFunction; + +/** + * Reduce a collection using an initial state and function + */ +@UdfDescription( + name = "reduce", + category = FunctionCategory.LAMBDA, + description = "Reduce the input collection down to a single value " + + "using an initial state and a function. " + + "The initial state (s) is passed into the scope of the function. " + + "Each invocation returns a new value for s, " + + "which the next invocation will receive. " + + "The final value for s is returned.", + author = KsqlConstants.CONFLUENT_AUTHOR +) +public class Reduce { + + @Udf(description = "When reducing an array, " + + "the reduce function must have two arguments. " + + "The two arguments for the reduce function are in order: " + + "the state and the array item. " + + "The final state is returned." + ) + public S reduceArray( + @UdfParameter(description = "The initial state.") final S initialState, + @UdfParameter(description = "The array.") final List list, + @UdfParameter(description = "The reduce function.") final BiFunction biFunction + ) { + if (initialState == null) { + return null; + } + + if (list == null) { + return initialState; + } + + S state = initialState; + for (T listItem: list) { + state = biFunction.apply(state, listItem); + } + return state; + } + + @Udf(description = "When reducing a map, " + + "the reduce function must have three arguments. " + + "The three arguments for the reduce function are in order: " + + "the state, the key, and the value. " + + "The final state is returned." + ) + public S reduceMap( + @UdfParameter(description = "The initial state.") final S initialState, + @UdfParameter(description = "The map.") final Map map, + @UdfParameter(description = "The reduce function.") final TriFunction triFunction + ) { + if (initialState == null) { + return null; + } + + if (map == null) { + return initialState; + } + + S state = initialState; + for (Entry entry : map.entrySet()) { + state = triFunction.apply(state, entry.getKey(), entry.getValue()); + } + return state; + } +} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapTransform.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Transform.java similarity index 52% rename from ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapTransform.java rename to ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Transform.java index d81ba78cae49..f7a1de2a1549 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapTransform.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/lambda/Transform.java @@ -13,35 +13,56 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.function.udf.map; +package io.confluent.ksql.function.udf.lambda; import io.confluent.ksql.function.FunctionCategory; 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.util.List; import java.util.Map; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.stream.Collectors; /** - * Transform a map's key and values using two lambda functions + * Transform a collection with a function */ @UdfDescription( - name = "map_transform", - category = FunctionCategory.MAP, - description = "Apply one function to each key and " - + "one function to each value of a map. " - + "The two arguments for each function are in order: key, value. " - + "The first function provided will be applied to each key and the " - + "second one applied to each value. " - + "The transformed map is returned.", + name = "transform", + category = FunctionCategory.LAMBDA, + description = "Apply a function to each element in a collection. " + + "The transformed collection is returned.", author = KsqlConstants.CONFLUENT_AUTHOR ) -public class MapTransform { +public class Transform { - @Udf - public Map mapTransform( + @Udf(description = "When transforming an array, " + + "the function provided must have two arguments. " + + "The two arguments for each function are in order: " + + "the key and then the value. " + + "The transformed array is returned." + ) + public List transformArray( + @UdfParameter(description = "The array") final List array, + @UdfParameter(description = "The lambda function") final Function function + ) { + if (array == null) { + return null; + } + return array.stream().map(function::apply).collect(Collectors.toList()); + } + + @Udf(description = "When transforming a map, " + + "two functions must be provided. " + + "For each map entry, the first function provided will " + + "be applied to the key and the second one applied to the value. " + + "Each function must have two arguments. " + + "The two arguments for each function are in order: the key and then the value. " + + "The transformed map is returned." + ) + public Map transformMap( @UdfParameter(description = "The map") final Map map, @UdfParameter(description = "The key lambda function") final BiFunction biFunction1, @UdfParameter(description = "The value lambda function") final BiFunction biFunction2 diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/ReduceMap.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/ReduceMap.java deleted file mode 100644 index 5bace9f9dbac..000000000000 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/ReduceMap.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright 2021 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.map; - -import io.confluent.ksql.execution.codegen.helpers.TriFunction; -import io.confluent.ksql.function.FunctionCategory; -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.util.Map; -import java.util.Map.Entry; - -/** - * Reduce a map using an initial state and function - */ -@UdfDescription( - name = "reduce_map", - category = FunctionCategory.MAP, - description = "Reduce the input map down to a single value " - + "using an initial state and a function. " - + "The initial state (s) is passed into the scope of the function. " - + "Each invocation returns a new value for s, " - + "which the next invocation will receive. " - + "The final value for s is returned." - + "The three arguments for the function are in order: key, value, state.", - author = KsqlConstants.CONFLUENT_AUTHOR -) -public class ReduceMap { - - @Udf - public S reduceMap( - @UdfParameter(description = "The map") final Map map, - @UdfParameter(description = "The initial state") final S initialState, - @UdfParameter(description = "The reduce function") final TriFunction triFunction - ) { - if (map == null) { - return null; - } - - S state = initialState; - for (Entry entry : map.entrySet()) { - state = triFunction.apply(entry.getKey(), entry.getValue(), state); - } - return state; - } -} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/UdfLoaderTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/UdfLoaderTest.java index 105b005a6260..eb8353231f1a 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/function/UdfLoaderTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/UdfLoaderTest.java @@ -203,15 +203,15 @@ public void shouldLoadLambdaReduceUdfs() { SqlTypes.INTEGER); // When: - final KsqlScalarFunction fun = FUNC_REG.getUdfFactory(FunctionName.of("reduce_map")) + final KsqlScalarFunction fun = FUNC_REG.getUdfFactory(FunctionName.of("reduce")) .getFunction( ImmutableList.of( - SqlArgument.of(SqlMap.of(SqlTypes.INTEGER, SqlTypes.INTEGER)), SqlArgument.of(SqlTypes.INTEGER), + SqlArgument.of(SqlMap.of(SqlTypes.INTEGER, SqlTypes.INTEGER)), SqlArgument.of(lambda))); // Then: - assertThat(fun.name().text(), equalToIgnoringCase("reduce_map")); + assertThat(fun.name().text(), equalToIgnoringCase("reduce")); } @Test @@ -223,14 +223,14 @@ public void shouldLoadLambdaTransformUdfs() { SqlTypes.INTEGER); // When: - final KsqlScalarFunction fun = FUNC_REG.getUdfFactory(FunctionName.of("array_transform")) + final KsqlScalarFunction fun = FUNC_REG.getUdfFactory(FunctionName.of("transform")) .getFunction( ImmutableList.of( SqlArgument.of(SqlArray.of(SqlTypes.INTEGER)), SqlArgument.of(lambda))); // Then: - assertThat(fun.name().text(), equalToIgnoringCase("array_transform")); + assertThat(fun.name().text(), equalToIgnoringCase("transform")); } @Test diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayTransformTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayTransformTest.java deleted file mode 100644 index adeda725aadb..000000000000 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayTransformTest.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Copyright 2021 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 java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.function.Function; -import org.junit.Before; -import org.junit.Test; - -public class ArrayTransformTest { - - private ArrayTransform udf; - - @Before - public void setUp() { - udf = new ArrayTransform(); - } - - @Test - public void shouldReturnNullForNullArray() { - assertThat(udf.arrayTransform(null, function1()), is(nullValue())); - } - - @Test - public void shouldApplyFunctionToEachElement() { - assertThat(udf.arrayTransform(Collections.emptyList(), function1()), is(Collections.emptyList())); - assertThat(udf.arrayTransform(Arrays.asList(-5, -2, 0), function1()), is(Arrays.asList(0, 3, 5))); - assertThat(udf.arrayTransform(Arrays.asList(3, null, 5), function1()), is(Arrays.asList(8, null, 10))); - - assertThat(udf.arrayTransform(Collections.emptyList(), function2()), is(Collections.emptyList())); - assertThat(udf.arrayTransform(Arrays.asList(-5, -2, 0), function2()), is(Arrays.asList("odd", "even", "even"))); - assertThat(udf.arrayTransform(Arrays.asList(3, null, 5), function2()), is(Arrays.asList("odd", null, "odd"))); - - assertThat(udf.arrayTransform(Collections.emptyList(), function3()), is(Collections.emptyList())); - assertThat(udf.arrayTransform(Arrays.asList("steven", "leah"), function3()), is(Arrays.asList("hello steven", "hello leah"))); - assertThat(udf.arrayTransform(Arrays.asList("rohan", null, "almog"), function3()), is(Arrays.asList("hello rohan", null, "hello almog"))); - - assertThat(udf.arrayTransform(Collections.emptyList(), function4()), is(Collections.emptyList())); - assertThat(udf.arrayTransform(Arrays.asList(Arrays.asList(5, 4 ,3), Collections.emptyList()), function4()), is(Arrays.asList(3, 0))); - assertThat(udf.arrayTransform(Arrays.asList(Arrays.asList(334, 1), null), function4()), is(Arrays.asList(2, null))); - } - - private Function function1() { - return x -> x + 5; - } - - private Function function2() { - return x -> { - if(x % 2 == 0) { - return "even"; - } else { - return "odd"; - } - }; - } - - private Function function3() { - return "hello "::concat; - } - - private Function, Integer> function4() { - return List::size; - } -} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java new file mode 100644 index 000000000000..4967ea732919 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2021 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.lambda; + +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 com.google.common.collect.ImmutableList; +import io.confluent.ksql.execution.codegen.helpers.TriFunction; +import io.confluent.ksql.function.GenericsUtil; +import io.confluent.ksql.util.KsqlException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; +import org.junit.Before; +import org.junit.Test; + +public class ReduceTest { + + private Reduce udf; + + @Before + public void setUp() { + udf = new Reduce(); + } + + @Test + public void shouldReturnOriginalStateForNullCollection() { + assertThat(udf.reduceMap(0, null, triFunction1()), is(0)); + assertThat(udf.reduceArray("", null, biFunction1()), is("")); + } + + @Test + public void shouldReturnNullForNullState() { + assertThat(udf.reduceMap(null, Collections.emptyMap(), triFunction1()), is(nullValue())); + assertThat(udf.reduceArray(null, Collections.emptyList(), biFunction1()), is(nullValue())); + } + + @Test + public void shouldReduceMap() { + final Map map1 = new HashMap<>(); + assertThat(udf.reduceMap(3, map1, triFunction1()), is(3)); + map1.put(4, 3); + map1.put(6, 2); + assertThat(udf.reduceMap(42, map1,triFunction1()), is(57)); + assertThat(udf.reduceMap(-4, map1, triFunction1()), is(11)); + map1.put(0,0); + assertThat(udf.reduceMap(0, map1, triFunction1()), is(15)); + + final Map map2 = new HashMap<>(); + assertThat(udf.reduceMap("", map2, triFunction2()), is("")); + map2.put("a", 42); + map2.put("b", 11); + assertThat(udf.reduceMap("", map2, triFunction2()), is("ba")); + assertThat(udf.reduceMap("string", map2, triFunction2()), is("bastring")); + map2.put("c",0); + map2.put("d",15); + map2.put("e",-5); + assertThat(udf.reduceMap("q", map2, triFunction2()), is("dbaq")); + } + + @Test + public void shouldReduceArray() { + assertThat(udf.reduceArray("", ImmutableList.of(), biFunction1()), is("")); + assertThat(udf.reduceArray("answer", ImmutableList.of(), biFunction1()), is("answer")); + assertThat(udf.reduceArray("", ImmutableList.of(2, 3, 4, 4, 1000), biFunction1()), is("evenoddeveneveneven")); + assertThat(udf.reduceArray("This is: ", ImmutableList.of(3, -1, -5), biFunction1()), is("This is: oddoddodd")); + + assertThat(udf.reduceArray(0, ImmutableList.of(), biFunction2()), is(0)); + assertThat(udf.reduceArray(14, Arrays.asList(-1, -13), biFunction2()), is(0)); + assertThat(udf.reduceArray(1, ImmutableList.of(-5, 10), biFunction2()), is(6)); + assertThat(udf.reduceArray(-100, ImmutableList.of(100, 1000, 42), biFunction2()), is(1042)); + } + + @Test + public void shouldNotSkipNullValuesWhenReducing() { + assertThrows( + NullPointerException.class, + () -> udf.reduceArray(0, Collections.singletonList(null), biFunction2()) + ); + assertThrows( + NullPointerException.class, + () -> udf.reduceArray(14, Arrays.asList(-1, -13, null), biFunction2()) + ); + + final Map map1 = new HashMap<>(); + map1.put(4, 3); + map1.put(6, null); + assertThrows( + NullPointerException.class, + () -> udf.reduceMap(3, map1, triFunction1()) + ); + } + + private TriFunction triFunction1() { + return (x,y,z) -> x + y + z; + } + + private TriFunction triFunction2() { + return (x, y, z) -> { + if(z - 10 > 0) { + return y.concat(x); + } + return x; + }; + } + + private BiFunction biFunction1() { + return (x,y) -> { + if (y % 2 == 0) { + return x.concat("even"); + } else { + return x.concat("odd"); + } + }; + } + + private BiFunction biFunction2() { + return Integer::sum; + } +} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/TransformTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/TransformTest.java new file mode 100644 index 000000000000..eb2740810970 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/TransformTest.java @@ -0,0 +1,152 @@ +/* + * Copyright 2021 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.lambda; + +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 java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.junit.Before; +import org.junit.Test; + +public class TransformTest { + + private Transform udf; + + @Before + public void setUp() { + udf = new Transform(); + } + + @Test + public void shouldReturnNullForNullArray() { + assertThat(udf.transformArray(null, function1()), is(nullValue())); + assertThat(udf.transformMap(null, biFunction1(), biFunction2()), is(nullValue())); + } + + @Test + public void shouldReturnTransformedArray() { + assertThat(udf.transformArray(Collections.emptyList(), function1()), is(Collections.emptyList())); + assertThat(udf.transformArray(Arrays.asList(-5, -2, 0), function1()), is(Arrays.asList(0, 3, 5))); + + assertThat(udf.transformArray(Collections.emptyList(), function2()), is(Collections.emptyList())); + assertThat(udf.transformArray(Arrays.asList(-5, -2, 0), function2()), is(Arrays.asList("odd", "even", "even"))); + + assertThat(udf.transformArray(Collections.emptyList(), function3()), is(Collections.emptyList())); + assertThat(udf.transformArray(Arrays.asList("steven", "leah"), function3()), is(Arrays.asList("hello steven", "hello leah"))); + + assertThat(udf.transformArray(Collections.emptyList(), function4()), is(Collections.emptyList())); + assertThat(udf.transformArray(Arrays.asList(Arrays.asList(5, 4 ,3), Collections.emptyList()), function4()), is(Arrays.asList(3, 0))); + } + + @Test + public void shouldReturnTransformedMap() { + final Map map1 = new HashMap<>(); + assertThat(udf.transformMap(map1, biFunction1(), biFunction2()), is(Collections.emptyMap())); + map1.put(3, 100); + map1.put(1, -2); + assertThat(udf.transformMap(map1, biFunction1(), biFunction2()), is(Stream.of(new Object[][] { + { -97, 97 }, + { 3, -3 }, + }).collect(Collectors.toMap(data -> (Integer) data[0], data -> (Integer) data[1])))); + + final Map map2 = new HashMap<>(); + assertThat(udf.transformMap(map2, biFunction3(), biFunction4()), is(Collections.emptyMap())); + map2.put("123", "456789"); + map2.put("hello", "hi"); + assertThat(udf.transformMap(map2, biFunction3(), biFunction4()), is(Stream.of(new Object[][] { + { "456789123", false }, + { "hihello", true }, + }).collect(Collectors.toMap(data -> (String) data[0], data -> (Boolean) data[1])))); + } + + + @Test + public void shouldNotSkipNullValuesWhenTransforming() { + assertThrows( + NullPointerException.class, + () -> udf.transformArray(Arrays.asList(Arrays.asList(334, 1), null), function4()) + ); + assertThrows( + NullPointerException.class, + () -> udf.transformArray(Arrays.asList("rohan", null, "almog"), function3()) + ); + assertThrows( + NullPointerException.class, + () -> udf.transformArray(Arrays.asList(3, null, 5), function2()) + ); + assertThrows( + NullPointerException.class, + () -> udf.transformArray(Arrays.asList(3, null, 5), function1()) + ); + + final Map map1 = new HashMap<>(); + map1.put(4, 3); + map1.put(6, null); + assertThrows( + NullPointerException.class, + () -> udf.transformMap(map1, biFunction1(), biFunction2()) + ); + } + + private Function function1() { + return x -> x + 5; + } + + private Function function2() { + return x -> { + if(x % 2 == 0) { + return "even"; + } else { + return "odd"; + } + }; + } + + private Function function3() { + return "hello "::concat; + } + + private Function, Integer> function4() { + return List::size; + } + + private BiFunction biFunction1() { + return (x,y) -> x - y; + } + + private BiFunction biFunction2() { + return (x,y) -> y - x; + } + + private BiFunction biFunction3() { + return (x,y) -> y.concat(x); + } + + private BiFunction biFunction4() { + return (x,y) -> x.length() > y.length(); + } +} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapTransformTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapTransformTest.java deleted file mode 100644 index 2f8cea314ac6..000000000000 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapTransformTest.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright 2021 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.map; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.function.BiFunction; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.junit.Before; -import org.junit.Test; - -public class MapTransformTest { - - private MapTransform udf; - - @Before - public void setUp() { - udf = new MapTransform(); - } - - @Test - public void shouldReturnNullForNullMap() { - assertThat(udf.mapTransform(null, biFunction1(), biFunction2()), is(nullValue())); - } - - @Test - public void shouldReturnTransformedMap() { - final Map map1 = new HashMap<>(); - assertThat(udf.mapTransform(map1, biFunction1(), biFunction2()), is(Collections.emptyMap())); - map1.put(3, 100); - map1.put(1, -2); - assertThat(udf.mapTransform(map1, biFunction1(), biFunction2()), is(Stream.of(new Object[][] { - { -97, 97 }, - { 3, -3 }, - }).collect(Collectors.toMap(data -> (Integer) data[0], data -> (Integer) data[1])))); - - final Map map2 = new HashMap<>(); - assertThat(udf.mapTransform(map2, biFunction3(), biFunction4()), is(Collections.emptyMap())); - map2.put("123", "456789"); - map2.put("hello", "hi"); - assertThat(udf.mapTransform(map2, biFunction3(), biFunction4()), is(Stream.of(new Object[][] { - { "456789123", false }, - { "hihello", true }, - }).collect(Collectors.toMap(data -> (String) data[0], data -> (Boolean) data[1])))); - } - - private BiFunction biFunction1() { - return (x,y) -> x - y; - } - - private BiFunction biFunction2() { - return (x,y) -> y - x; - } - - private BiFunction biFunction3() { - return (x,y) -> y.concat(x); - } - - private BiFunction biFunction4() { - return (x,y) -> x.length() > y.length(); - } -} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/ReduceMapTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/ReduceMapTest.java deleted file mode 100644 index 5253485e7e5c..000000000000 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/ReduceMapTest.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright 2021 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.map; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; - -import io.confluent.ksql.execution.codegen.helpers.TriFunction; -import java.util.HashMap; -import java.util.Map; -import org.junit.Before; -import org.junit.Test; - -public class ReduceMapTest { - - private ReduceMap udf; - - @Before - public void setUp() { - udf = new ReduceMap(); - } - - @Test - public void shouldReturnNullForNullMap() { - assertThat(udf.reduceMap(null, 0, triFunction1()), is(nullValue())); - } - - @Test - public void shouldReduceMap() { - final Map map1 = new HashMap<>(); - assertThat(udf.reduceMap(map1, 3, triFunction1()), is(3)); - map1.put(4, 3); - map1.put(6, 2); - assertThat(udf.reduceMap(map1, 42, triFunction1()), is(57)); - assertThat(udf.reduceMap(map1, -4, triFunction1()), is(11)); - map1.put(0,0); - assertThat(udf.reduceMap(map1, 0, triFunction1()), is(15)); - - final Map map2 = new HashMap<>(); - assertThat(udf.reduceMap(map2, "", triFunction2()), is("")); - map2.put("a", 42); - map2.put("b", 11); - assertThat(udf.reduceMap(map2, "", triFunction2()), is("ba")); - assertThat(udf.reduceMap(map2, "string", triFunction2()), is("bastring")); - map2.put("c",0); - map2.put("d",15); - map2.put("e",-5); - assertThat(udf.reduceMap(map2, "q", triFunction2()), is("dbaq")); - } - - private TriFunction triFunction1() { - return (x,y,z) -> x + y + z; - } - - private TriFunction triFunction2() { - return (x, y, z) -> { - if(y - 10 > 0) { - return x.concat(z); - } - return z; - }; - } -} \ No newline at end of file diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/util/ExpressionTypeManager.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/util/ExpressionTypeManager.java index ae24c19c0140..688a150838f6 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/util/ExpressionTypeManager.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/util/ExpressionTypeManager.java @@ -319,7 +319,7 @@ public Void visitSearchedCaseExpression( final Optional whenType = validateWhenClauses(node.getWhenClauses(), context); final Optional defaultType = node.getDefaultValue() - .map(ExpressionTypeManager.this::getExpressionSqlType); + .map(expression -> getExpressionSqlType(expression, context)); if (whenType.isPresent() && defaultType.isPresent()) { if (!whenType.get().equals(defaultType.get())) { diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/plan.json new file mode 100644 index 000000000000..da95d06983e0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, NUMBERS ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "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 REDUCE(2, TEST.NUMBERS, (S, X) => (S + X)) REDUCE\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "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" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `NUMBERS` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "REDUCE(2, NUMBERS, (S, X) => (S + X)) AS REDUCE" ] + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/spec.json new file mode 100644 index 000000000000..53c612cf6238 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/spec.json @@ -0,0 +1,120 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614136896616, + "path" : "query-validation-tests/reduce-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "reduce an array", + "inputs" : [ { + "topic" : "test_topic", + "key" : "one", + "value" : { + "numbers" : [ 3, 6 ] + } + }, { + "topic" : "test_topic", + "key" : "two", + "value" : { + "numbers" : [ 5, null ] + } + }, { + "topic" : "test_topic", + "key" : "three", + "value" : { + "numbers" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "one", + "value" : { + "REDUCE" : 11 + } + }, { + "topic" : "OUTPUT", + "key" : "two", + "value" : { + "REDUCE" : null + } + }, { + "topic" : "OUTPUT", + "key" : "three", + "value" : { + "REDUCE" : 2 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM test (ID STRING KEY, numbers ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(2, numbers, (s, x) => s + x) AS reduce FROM test;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array/7.0.0_1614136896616/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/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/plan.json new file mode 100644 index 000000000000..6a446497fc6f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, NUMBERS ARRAY, STATE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY, `STATE` 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 REDUCE(TEST.STATE, TEST.NUMBERS, (S, X) => (S + X)) REDUCE\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `REDUCE` 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" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `NUMBERS` ARRAY, `STATE` BIGINT" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "REDUCE(STATE, NUMBERS, (S, X) => (S + X)) AS REDUCE" ] + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/spec.json new file mode 100644 index 000000000000..347aa525aed0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614136896674, + "path" : "query-validation-tests/reduce-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY, `STATE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `REDUCE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "reduce an array with null initial state", + "inputs" : [ { + "topic" : "test_topic", + "key" : "one", + "value" : { + "numbers" : [ 1, 2 ], + "state" : null + } + }, { + "topic" : "test_topic", + "key" : "two", + "value" : { + "numbers" : null, + "state" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "one", + "value" : { + "REDUCE" : null + } + }, { + "topic" : "OUTPUT", + "key" : "two", + "value" : { + "REDUCE" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM test (ID STRING KEY, numbers ARRAY, state BIGINT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(state, numbers, (s, x) => s + x) AS reduce FROM test;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `REDUCE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY, `STATE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-array_-_reduce_an_array_with_null_initial_state/7.0.0_1614136896674/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/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/plan.json new file mode 100644 index 000000000000..bea4fc79b3e6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, MAP MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `MAP` MAP", + "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 REDUCE(0, TEST.MAP, (S, K, V) => (CASE WHEN (LEN(K) > 3) THEN (S + V) ELSE (S - V) END)) REDUCE\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "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" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `MAP` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "REDUCE(0, MAP, (S, K, V) => (CASE WHEN (LEN(K) > 3) THEN (S + V) ELSE (S - V) END)) AS REDUCE" ] + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/spec.json new file mode 100644 index 000000000000..c86af57a3764 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/spec.json @@ -0,0 +1,128 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614069675859, + "path" : "query-validation-tests/reduce-map.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `MAP` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "apply reduce lambda function to map", + "inputs" : [ { + "topic" : "test_topic", + "key" : "zero", + "value" : { + "map" : { + "123" : 3, + "12" : 7, + "1234" : 2 + } + } + }, { + "topic" : "test_topic", + "key" : "one", + "value" : { + "map" : { + "1" : 1, + "ttttt" : null, + "" : 3 + } + } + }, { + "topic" : "test_topic", + "key" : "two", + "value" : { + "map" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "zero", + "value" : { + "REDUCE" : -8 + } + }, { + "topic" : "OUTPUT", + "key" : "one", + "value" : { + "REDUCE" : null + } + }, { + "topic" : "OUTPUT", + "key" : "two", + "value" : { + "REDUCE" : 0 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM test (ID STRING KEY, map MAP) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(0, map, (s, k, v) => CASE WHEN LEN(k) > 3 THEN s + v ELSE s - v END) AS reduce FROM test;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `REDUCE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `MAP` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_apply_reduce_lambda_function_to_map/7.0.0_1614069675859/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/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/plan.json new file mode 100644 index 000000000000..6e24305b14b4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, MAP MAP, STATE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `MAP` MAP, `STATE` 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 REDUCE(TEST.STATE, TEST.MAP, (S, X, Y) => (Y + S)) REDUCE\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `REDUCE` 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" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `MAP` MAP, `STATE` BIGINT" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "REDUCE(STATE, MAP, (S, X, Y) => (Y + S)) AS REDUCE" ] + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/spec.json new file mode 100644 index 000000000000..787d64f5a4d1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/spec.json @@ -0,0 +1,113 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614069675904, + "path" : "query-validation-tests/reduce-map.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `MAP` MAP, `STATE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `REDUCE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "reduce map with null initial state", + "inputs" : [ { + "topic" : "test_topic", + "key" : "one", + "value" : { + "map" : { + "test1" : 6, + "test2" : 7 + }, + "state" : null + } + }, { + "topic" : "test_topic", + "key" : "two", + "value" : { + "map" : null, + "state" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "one", + "value" : { + "REDUCE" : null + } + }, { + "topic" : "OUTPUT", + "key" : "two", + "value" : { + "REDUCE" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM test (ID STRING KEY, map MAP, state BIGINT) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(state, map, (s,x,y) => y + s) AS reduce FROM test;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `REDUCE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `MAP` MAP, `STATE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/reduce-map_-_reduce_map_with_null_initial_state/7.0.0_1614069675904/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/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/plan.json new file mode 100644 index 000000000000..d07b19dc8d53 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, NUMBERS ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "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 TRANSFORM(TEST.NUMBERS, (X) => (X + 5)) C\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `C` ARRAY", + "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" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `NUMBERS` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(NUMBERS, (X) => (X + 5)) AS C" ] + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/spec.json new file mode 100644 index 000000000000..b2f0df5ca285 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/spec.json @@ -0,0 +1,120 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614069966647, + "path" : "query-validation-tests/transform-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `C` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "apply transform lambda function to array", + "inputs" : [ { + "topic" : "test_topic", + "key" : "one", + "value" : { + "numbers" : [ 3, 6 ] + } + }, { + "topic" : "test_topic", + "key" : "two", + "value" : { + "numbers" : [ 5, null ] + } + }, { + "topic" : "test_topic", + "key" : "three", + "value" : { + "numbers" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "one", + "value" : { + "C" : [ 8, 11 ] + } + }, { + "topic" : "OUTPUT", + "key" : "two", + "value" : { + "C" : null + } + }, { + "topic" : "OUTPUT", + "key" : "three", + "value" : { + "C" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM test (ID STRING KEY, numbers ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, TRANSFORM(numbers, x => x + 5) AS c FROM test;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `C` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `NUMBERS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_apply_transform_lambda_function_to_array/7.0.0_1614069966647/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/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/plan.json new file mode 100644 index 000000000000..bd610aa5b2fe --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY>) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X) => ARRAY_MAX(X)) MAX\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `MAX` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` ARRAY>" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X) => ARRAY_MAX(X)) AS MAX" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/spec.json new file mode 100644 index 000000000000..9258bf07134c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/spec.json @@ -0,0 +1,180 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614070842316, + "path" : "query-validation-tests/transform-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `MAX` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "array max on array of arrays", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : [ [ 5, 7, 1 ], [ 3, 6, 1 ] ] + } + }, { + "topic" : "test_topic", + "key" : 5, + "value" : { + "value" : [ [ 123, 452, 451, null ], [ 532, 123, 78 ] ] + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : [ [ 90, 341, 2 ], [ 234, 123, 865 ] ] + } + }, { + "topic" : "test_topic", + "key" : 110, + "value" : { + "value" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "MAX" : [ 7, 6 ] + } + }, { + "topic" : "OUTPUT", + "key" : 5, + "value" : { + "MAX" : [ 452, 532 ] + } + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "MAX" : [ 341, 865 ] + } + }, { + "topic" : "OUTPUT", + "key" : 110, + "value" : { + "MAX" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY>) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => array_max(x)) as max from TEST emit changes;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `MAX` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "MAX", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_array_max_on_array_of_arrays/7.0.0_1614070842316/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/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/plan.json new file mode 100644 index 000000000000..d488e4f081e3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X) => UCASE(X)) LAMBDA\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `LAMBDA` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X) => UCASE(X)) AS LAMBDA" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/spec.json new file mode 100644 index 000000000000..cf8014a2911c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614070716263, + "path" : "query-validation-tests/transform-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `LAMBDA` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "capitalize all array elements", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : [ "hello", "these", "are", "my", "strings" ] + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "VALUE" : [ "check", null, "null" ] + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "VALUE" : [ "ksqldb", "kafka", "streams" ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "LAMBDA" : [ "HELLO", "THESE", "ARE", "MY", "STRINGS" ] + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "LAMBDA" : [ "CHECK", null, "NULL" ] + } + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "LAMBDA" : [ "KSQLDB", "KAFKA", "STREAMS" ] + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => UCASE(x)) AS LAMBDA FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `LAMBDA` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "LAMBDA", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_capitalize_all_array_elements/7.0.0_1614070716263/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/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/plan.json new file mode 100644 index 000000000000..9087cf97b64c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X) => (CASE WHEN (X > 10) THEN 'above 10' ELSE 'TOO LOW' END)) SUM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `SUM` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X) => (CASE WHEN (X > 10) THEN 'above 10' ELSE 'TOO LOW' END)) AS SUM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/spec.json new file mode 100644 index 000000000000..9bc76f4fcdb6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/spec.json @@ -0,0 +1,174 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614070842246, + "path" : "query-validation-tests/transform-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `SUM` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "case check on all array elements - input: double, output: string", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : [ 2.32, 12123, 3.123, 4.45 ] + } + }, { + "topic" : "test_topic", + "key" : 5, + "value" : { + "value" : [ 11, 13, null ] + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : [ 10, 5, 4 ] + } + }, { + "topic" : "test_topic", + "key" : 110, + "value" : { + "value" : [ 2, 3, 100 ] + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "SUM" : [ "TOO LOW", "above 10", "TOO LOW", "TOO LOW" ] + } + }, { + "topic" : "OUTPUT", + "key" : 5, + "value" : { + "SUM" : [ "above 10", "above 10", "TOO LOW" ] + } + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "SUM" : [ "TOO LOW", "TOO LOW", "TOO LOW" ] + } + }, { + "topic" : "OUTPUT", + "key" : 110, + "value" : { + "SUM" : [ "TOO LOW", "TOO LOW", "above 10" ] + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "double" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => CASE WHEN x > 10 THEN 'above 10' ELSE 'TOO LOW' END) as SUM from TEST emit changes;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `SUM` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "double" ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "SUM", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "string" ] + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_case_check_on_all_array_elements_-_input__double,_output__string/7.0.0_1614070842246/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/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/plan.json new file mode 100644 index 000000000000..07c57980421e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X) => ABS(X)) ABS\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `ABS` ARRAY", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` ARRAY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X) => ABS(X)) AS ABS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/spec.json new file mode 100644 index 000000000000..14a3682c138d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/spec.json @@ -0,0 +1,201 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614070842293, + "path" : "query-validation-tests/transform-array.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `ABS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "decimal absolute on all array elements", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : [ -2.45, 3.67, 1.23 ] + } + }, { + "topic" : "test_topic", + "key" : 5, + "value" : { + "value" : [ -7.45, -1.34, null ] + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : [ 1.45, 5.68, -4.67 ] + } + }, { + "topic" : "test_topic", + "key" : 110, + "value" : { + "value" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "ABS" : [ 2.45, 3.67, 1.23 ] + } + }, { + "topic" : "OUTPUT", + "key" : 5, + "value" : { + "ABS" : [ 7.45, 1.34, null ] + } + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "ABS" : [ 1.45, 5.68, 4.67 ] + } + }, { + "topic" : "OUTPUT", + "key" : 110, + "value" : { + "ABS" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 3, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "2", + "connect.decimal.precision" : "3" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => abs(x)) as abs from TEST emit changes;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `ABS` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 3, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "2", + "connect.decimal.precision" : "3" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ] + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ABS", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 3, + "logicalType" : "decimal" + } ] + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-array_-_decimal_absolute_on_all_array_elements/7.0.0_1614070842293/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/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/plan.json new file mode 100644 index 000000000000..3d477fc57874 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X, Y) => UCASE(X), (X, Y) => ROUND(Y)) TRANSFORM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` MAP" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X, Y) => UCASE(X), (X, Y) => ROUND(Y)) AS TRANSFORM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/spec.json new file mode 100644 index 000000000000..7fe251452cc4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/spec.json @@ -0,0 +1,230 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614073059361, + "path" : "query-validation-tests/transform-map.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "capitalize all keys and round values in transformed map", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : { + "test" : 3.21, + "hello" : 4.49 + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "VALUE" : { + "number" : 10.50, + "other" : 1.01 + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "TRANSFORM" : { + "TEST" : 3, + "HELLO" : 4 + } + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "TRANSFORM" : { + "NUMBER" : 11, + "OTHER" : 1 + } + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 4, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "2", + "connect.decimal.precision" : "4" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => UCASE(x), (x,y) => round(y)) AS transform FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 4, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "2", + "connect.decimal.precision" : "4" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "TRANSFORM", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_TRANSFORM", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "bytes", + "scale" : 0, + "precision" : 2, + "logicalType" : "decimal" + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + } + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_capitalize_all_keys_and_round_values_in_transformed_map/7.0.0_1614073059361/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/transform-map_-_transform_a_map/7.0.0_1614072121318/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/plan.json new file mode 100644 index 000000000000..c3d5ddb08dd2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X, Y) => (X + '_test'), (K, V) => (ARRAY_MIN(V) + 1)) TRANSFORM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` MAP>" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X, Y) => (X + '_test'), (K, V) => (ARRAY_MIN(V) + 1)) AS TRANSFORM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/spec.json new file mode 100644 index 000000000000..a5fc9648cefc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/spec.json @@ -0,0 +1,217 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614072121318, + "path" : "query-validation-tests/transform-map.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "transform a map", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : { + "a" : [ 2, 4, 5 ], + "b" : [ -1, -2 ] + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : { + "q" : [ ] + } + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "value" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "TRANSFORM" : { + "a_test" : 3, + "b_test" : -1 + } + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "TRANSFORM" : null + } + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "TRANSFORM" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => x + '_test' , (k,v) => ARRAY_MIN(v) + 1) as transform from TEST emit changes;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "TRANSFORM", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_TRANSFORM", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + } + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transform_a_map/7.0.0_1614072121318/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/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/plan.json new file mode 100644 index 000000000000..bdf2507f37a7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/plan.json @@ -0,0 +1,145 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.ID ID,\n TRANSFORM(TEST.VALUE, (X, Y) => CAST(ARRAY_MAX(Y) AS STRING), (X, Y) => Y) TRANSFORM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP>", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "orReplace" : false + }, + "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" : "AVRO" + } + }, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` MAP>" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "TRANSFORM(VALUE, (X, Y) => CAST(ARRAY_MAX(Y) AS STRING), (X, Y) => Y) AS TRANSFORM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "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.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.variable.substitution.enable" : "true", + "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" : "true", + "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.query.transient.max.bytes.buffering.total" : "-1", + "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.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "false", + "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.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.query.pull.thread.pool.size" : "100", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/spec.json new file mode 100644 index 000000000000..42168f529592 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/spec.json @@ -0,0 +1,221 @@ +{ + "version" : "7.0.0", + "timestamp" : 1614072565004, + "path" : "query-validation-tests/transform-map.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "transformed map with duplicate keys ", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : { + "5" : [ 0, 1, 2 ], + "4" : [ -1, -2 ] + } + } + }, { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 2, + "value" : { + "value" : { + "1" : [ 1, 2 ], + "2" : [ -1, 2 ] + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "TRANSFORM" : { + "2" : [ 0, 1, 2 ], + "-1" : [ -1, -2 ] + } + } + }, { + "topic" : "OUTPUT", + "key" : 1, + "value" : { + "TRANSFORM" : null + } + }, { + "topic" : "OUTPUT", + "key" : 2, + "value" : { + "TRANSFORM" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => CAST(ARRAY_MAX(y) AS STRING), (x,y) => y) as transform from TEST emit changes;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `TRANSFORM` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` MAP>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_VALUE", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + }, + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema_VALUE" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "TRANSFORM", + "type" : [ "null", { + "type" : "array", + "items" : { + "type" : "record", + "name" : "KsqlDataSourceSchema_TRANSFORM", + "fields" : [ { + "name" : "key", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "value", + "type" : [ "null", { + "type" : "array", + "items" : [ "null", "int" ] + } ], + "default" : null + } ], + "connect.internal.type" : "MapEntry" + } + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/transform-map_-_transformed_map_with_duplicate_keys_/7.0.0_1614072565004/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/reduce-array.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-array.json new file mode 100644 index 000000000000..169605110c47 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-array.json @@ -0,0 +1,39 @@ +{ + "comments": [ + "Tests covering the use of the REDUCE function with arrays." + ], + "tests": [ + { + "name": "reduce an array", + "statements": [ + "CREATE STREAM test (ID STRING KEY, numbers ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(2, numbers, (s, x) => s + x) AS reduce FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": "one", "value": {"numbers": [3, 6]}}, + {"topic": "test_topic", "key": "two", "value": {"numbers": [5, null]}}, + {"topic": "test_topic", "key": "three", "value": {"numbers": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "one", "value": {"REDUCE": 11}}, + {"topic": "OUTPUT", "key": "two", "value": {"REDUCE": null}}, + {"topic": "OUTPUT", "key": "three", "value": {"REDUCE": 2}} + ] + }, + { + "name": "reduce an array with null initial state", + "statements": [ + "CREATE STREAM test (ID STRING KEY, numbers ARRAY, state BIGINT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(state, numbers, (s, x) => s + x) AS reduce FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": "one", "value": {"numbers": [1, 2], "state": null}}, + {"topic": "test_topic", "key": "two", "value": {"numbers": null, "state": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "one", "value": {"REDUCE": null}}, + {"topic": "OUTPUT", "key": "two", "value": {"REDUCE": null}} + ] + } + ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-map.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-map.json new file mode 100644 index 000000000000..11a532bf85fc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/reduce-map.json @@ -0,0 +1,39 @@ +{ + "comments": [ + "Tests covering the use of the REDUCE function with maps." + ], + "tests": [ + { + "name": "apply reduce lambda function to map", + "statements": [ + "CREATE STREAM test (ID STRING KEY, map MAP) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(0, map, (s, k, v) => CASE WHEN LEN(k) > 3 THEN s + v ELSE s - v END) AS reduce FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": "zero", "value": {"map": {"123": 3, "12": 7, "1234": 2}}}, + {"topic": "test_topic", "key": "one", "value": {"map": {"1": 1, "ttttt": null, "": 3}}}, + {"topic": "test_topic", "key": "two", "value": {"map": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "zero", "value": {"REDUCE": -8}}, + {"topic": "OUTPUT", "key": "one", "value": {"REDUCE": null}}, + {"topic": "OUTPUT", "key": "two", "value": {"REDUCE": 0}} + ] + }, + { + "name": "reduce map with null initial state", + "statements": [ + "CREATE STREAM test (ID STRING KEY, map MAP, state BIGINT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, REDUCE(state, map, (s,x,y) => y + s) AS reduce FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": "one", "value": {"map": {"test1": 6, "test2": 7}, "state": null}}, + {"topic": "test_topic", "key": "two", "value": {"map": null, "state": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "one", "value": {"REDUCE": null}}, + {"topic": "OUTPUT", "key": "two", "value": {"REDUCE": null}} + ] + } + ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-array.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-array.json new file mode 100644 index 000000000000..e0fb30adfcc8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-array.json @@ -0,0 +1,98 @@ +{ + "comments": [ + "Tests covering the use of the TRANSFORM function with arrays." + ], + "tests": [ + { + "name": "apply transform lambda function to array", + "statements": [ + "CREATE STREAM test (ID STRING KEY, numbers ARRAY) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT ID, TRANSFORM(numbers, x => x + 5) AS c FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": "one", "value": {"numbers": [3, 6]}}, + {"topic": "test_topic", "key": "two", "value": {"numbers": [5, null]}}, + {"topic": "test_topic", "key": "three", "value": {"numbers": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "one", "value": {"C":[8, 11]}}, + {"topic": "OUTPUT", "key": "two", "value": {"C":null}}, + {"topic": "OUTPUT", "key": "three", "value": {"C":null}} + ] + }, + { + "name": "capitalize all array elements", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => UCASE(x)) AS LAMBDA FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"VALUE": ["hello", "these", "are", "my", "strings"]}}, + {"topic": "test_topic", "key": 1, "value": {"VALUE": ["check", null, "null"]}}, + {"topic": "test_topic", "key": 2, "value": {"VALUE": ["ksqldb", "kafka", "streams"]}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"LAMBDA": ["HELLO", "THESE", "ARE", "MY", "STRINGS"]}}, + {"topic": "OUTPUT", "key": 1,"value": {"LAMBDA": ["CHECK", null, "NULL"]}}, + {"topic": "OUTPUT", "key": 2,"value": {"LAMBDA": ["KSQLDB", "KAFKA", "STREAMS"]}} + ] + }, + { + "name": "case check on all array elements - input: double, output: string", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => CASE WHEN x > 10 THEN 'above 10' ELSE 'TOO LOW' END) as SUM from TEST emit changes;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": [2.32, 12123, 3.123, 4.45]}}, + {"topic": "test_topic", "key": 5,"value": {"value": [11, 13, null]}}, + {"topic": "test_topic", "key": 100,"value": {"value": [10, 5, 4]}}, + {"topic": "test_topic", "key": 110,"value": {"value": [2, 3, 100]}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"SUM": ["TOO LOW", "above 10", "TOO LOW", "TOO LOW"]}}, + {"topic": "OUTPUT", "key": 5,"value": {"SUM": ["above 10", "above 10", "TOO LOW"]}}, + {"topic": "OUTPUT", "key": 100,"value": {"SUM": ["TOO LOW", "TOO LOW", "TOO LOW"]}}, + {"topic": "OUTPUT", "key": 110,"value": {"SUM": ["TOO LOW", "TOO LOW", "above 10"]}} + ] + }, + { + "name": "decimal absolute on all array elements", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => abs(x)) as abs from TEST emit changes;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": [-2.45, 3.67, 1.23]}}, + {"topic": "test_topic", "key": 5,"value": {"value": [-7.45, -1.34, null]}}, + {"topic": "test_topic", "key": 100,"value": {"value": [1.45, 5.68, -4.67]}}, + {"topic": "test_topic", "key": 110,"value": {"value": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"ABS": [2.45, 3.67, 1.23]}}, + {"topic": "OUTPUT", "key": 5,"value": {"ABS": [7.45, 1.34, null]}}, + {"topic": "OUTPUT", "key": 100,"value": {"ABS": [1.45, 5.68, 4.67]}}, + {"topic": "OUTPUT", "key": 110,"value": {"ABS": null}} + ] + }, + { + "name": "array max on array of arrays", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY>) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, x => array_max(x)) as max from TEST emit changes;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": [[5, 7, 1], [3, 6, 1]]}}, + {"topic": "test_topic", "key": 5,"value": {"value": [[123, 452, 451, null], [532, 123, 78]]}}, + {"topic": "test_topic", "key": 100,"value": {"value": [[90, 341, 2], [234, 123, 865]]}}, + {"topic": "test_topic", "key": 110,"value": {"value": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"MAX": [7, 6]}}, + {"topic": "OUTPUT", "key": 5,"value": {"MAX": [452, 532]}}, + {"topic": "OUTPUT", "key": 100,"value": {"MAX": [341, 865]}}, + {"topic": "OUTPUT", "key": 110,"value": {"MAX": null}} + ] + } + ] +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-map.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-map.json new file mode 100644 index 000000000000..8c880a4ed705 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/transform-map.json @@ -0,0 +1,56 @@ +{ + "comments": [ + "Tests covering the use of the TRANSFORM function with maps." + ], + "tests": [ + { + "name": "transform a map", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => x + '_test' , (k,v) => ARRAY_MIN(v) + 1) as transform from TEST emit changes;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": {"a": [2,4,5], "b": [-1,-2]}}}, + {"topic": "test_topic", "key": 1,"value": {"value": {"q": []}}}, + {"topic": "test_topic", "key": 2,"value": {"value": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"TRANSFORM": {"a_test": 3, "b_test": -1}}}, + {"topic": "OUTPUT", "key": 1,"value": {"TRANSFORM": null}}, + {"topic": "OUTPUT", "key": 2,"value": {"TRANSFORM": null}} + ] + }, + { + "name": "transformed map with duplicate keys ", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP>) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => CAST(ARRAY_MAX(y) AS STRING), (x,y) => y) as transform from TEST emit changes;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"VALUE": {"5": [0,1,2], "4": [-1,-2]}}}, + {"topic": "test_topic", "key": 1,"value": {"VALUE": null}}, + {"topic": "test_topic", "key": 2,"value": {"VALUE": {"1": [1,2], "2": [-1,2]}}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"TRANSFORM": {"2": [0,1,2], "-1": [-1,-2]}}}, + {"topic": "OUTPUT", "key": 1,"value": {"TRANSFORM": null}}, + {"topic": "OUTPUT", "key": 2,"value": {"TRANSFORM": null}} + ] + }, + { + "name": "capitalize all keys and round values in transformed map", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE MAP) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM(VALUE, (x,y) => UCASE(x), (x,y) => round(y)) AS transform FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"VALUE": {"test": 3.21, "hello": 4.49}}}, + {"topic": "test_topic", "key": 1, "value": {"VALUE": {"number": 10.50, "other": 1.01}}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": 0,"value": {"TRANSFORM": {"TEST": 3, "HELLO": 4}}}, + {"topic": "OUTPUT", "key": 1,"value": {"TRANSFORM":{"NUMBER": 11, "OTHER": 1}}} + ] + } + ] +} \ No newline at end of file diff --git a/ksqldb-udf/src/main/java/io/confluent/ksql/function/FunctionCategory.java b/ksqldb-udf/src/main/java/io/confluent/ksql/function/FunctionCategory.java index 7b6b69b557da..26c5f3fc17d4 100644 --- a/ksqldb-udf/src/main/java/io/confluent/ksql/function/FunctionCategory.java +++ b/ksqldb-udf/src/main/java/io/confluent/ksql/function/FunctionCategory.java @@ -32,4 +32,5 @@ private FunctionCategory() { public static final String OTHER = "OTHER"; public static final String AGGREGATE = "AGGREGATE"; public static final String TABLE = "TABLE"; + public static final String LAMBDA = "LAMBDA"; }