diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/GenericsUtil.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/GenericsUtil.java index 9d222f97d82e..a45dd33f31f4 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/function/GenericsUtil.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/GenericsUtil.java @@ -20,11 +20,13 @@ import com.google.common.collect.Sets; import io.confluent.ksql.function.types.ArrayType; import io.confluent.ksql.function.types.GenericType; +import io.confluent.ksql.function.types.LambdaType; import io.confluent.ksql.function.types.MapType; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.function.types.StructType; import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.schema.ksql.types.SqlArray; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlMap; import io.confluent.ksql.schema.ksql.types.SqlStruct.Builder; import io.confluent.ksql.schema.ksql.types.SqlType; @@ -34,6 +36,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -76,6 +79,14 @@ public static Set constituentGenerics(final ParamType type) { .collect(Collectors.toSet()); } else if (type instanceof GenericType) { return ImmutableSet.of(type); + } else if (type instanceof LambdaType) { + final Set inputSet = new HashSet<>(); + for (final ParamType paramType: ((LambdaType) type).inputTypes()) { + inputSet.addAll(constituentGenerics(paramType)); + } + return Sets.union( + inputSet, + constituentGenerics(((LambdaType) type).returnType())); } else { return ImmutableSet.of(); } @@ -172,11 +183,58 @@ public static Map resolveGenerics( return ImmutableMap.copyOf(mapping); } + public static Map resolveLambdaGenerics( + final ParamType schema, + final SqlLambda sqlLambda + ) { + final List> genericMapping = new ArrayList<>(); + boolean success; + + final LambdaType lambdaType = (LambdaType) schema; + boolean resolvedInputs = true; + if (sqlLambda.getInputType().size() != lambdaType.inputTypes().size()) { + throw new KsqlException( + "Number of lambda arguments don't match between schema and sql type"); + } + + int i = 0; + for (final ParamType paramType : lambdaType.inputTypes()) { + resolvedInputs = + resolvedInputs && resolveGenerics( + genericMapping, paramType, sqlLambda.getInputType().get(i)); + i++; + } + success = resolvedInputs && resolveGenerics(genericMapping, lambdaType.returnType(), sqlLambda.getReturnType()); + if (!success) { + throw new KsqlException( + String.format("Cannot infer generics for %s from %s because " + + "they do not have the same schema structure.", + schema, + sqlLambda)); + } + + final Map mapping = new HashMap<>(); + for (final Entry entry : genericMapping) { + final SqlType old = mapping.putIfAbsent(entry.getKey(), entry.getValue()); + if (old != null && !old.equals(entry.getValue())) { + throw new KsqlException(String.format( + "Found invalid instance of generic schema. Cannot map %s to both %s and %s", + schema, + old, + sqlLambda)); + } + } + + return ImmutableMap.copyOf(mapping); + } + + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private static boolean resolveGenerics( final List> mapping, final ParamType schema, final SqlType instance ) { + // CHECKSTYLE_RULES.ON: CyclomaticComplexity if (!isGeneric(schema) && !matches(schema, instance)) { // cannot identify from type mismatch return false; diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfFactory.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfFactory.java index 3547bbb74d46..15fcff66b625 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfFactory.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfFactory.java @@ -17,6 +17,7 @@ import io.confluent.ksql.function.udf.Kudf; import io.confluent.ksql.function.udf.UdfMetadata; +import io.confluent.ksql.schema.ksql.SqlArgument; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.util.KsqlException; import java.util.List; @@ -79,6 +80,10 @@ public String toString() { + '}'; } + public synchronized KsqlScalarFunction getUdfFunction(final List argTypes) { + return udfIndex.getUdfFunction(argTypes); + } + public synchronized KsqlScalarFunction getFunction(final List argTypes) { return udfIndex.getFunction(argTypes); } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfIndex.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfIndex.java index b66ce67d3e56..5bb4ac0caee1 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfIndex.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/UdfIndex.java @@ -19,8 +19,11 @@ import com.google.common.collect.Iterables; import io.confluent.ksql.function.types.ArrayType; import io.confluent.ksql.function.types.GenericType; +import io.confluent.ksql.function.types.LambdaType; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.function.types.ParamTypes; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.utils.FormatOptions; import io.confluent.ksql.util.KsqlException; @@ -142,6 +145,32 @@ void addFunction(final T function) { curr.update(function, order); } + T getUdfFunction(final List arguments) { + final List candidates = new ArrayList<>(); + + // first try to get the candidates without any implicit casting + getUdfCandidates(arguments, 0, root, candidates, new HashMap<>(), false); + final Optional fun = candidates + .stream() + .max(Node::compare) + .map(node -> node.value); + + if (fun.isPresent()) { + return fun.get(); + } else if (!supportsImplicitCasts) { + throw createNoMatchingFunctionExceptionSqlArgument(arguments); + } + + // if none were found (candidates is empty) try again with + // implicit casting + getUdfCandidates(arguments, 0, root, candidates, new HashMap<>(), true); + return candidates + .stream() + .max(Node::compare) + .map(node -> node.value) + .orElseThrow(() -> createNoMatchingFunctionExceptionSqlArgument(arguments)); + } + T getFunction(final List arguments) { final List candidates = new ArrayList<>(); @@ -168,6 +197,37 @@ T getFunction(final List arguments) { .orElseThrow(() -> createNoMatchingFunctionException(arguments)); } + private void getUdfCandidates( + final List arguments, + final int argIndex, + final Node current, + final List candidates, + final Map reservedGenerics, + final boolean allowCasts + ) { + if (argIndex == arguments.size()) { + if (current.value != null) { + candidates.add(current); + } + return; + } + final SqlType arg = arguments.get(argIndex).getSqlType(); + for (final Entry candidate : current.children.entrySet()) { + final Map reservedCopy = new HashMap<>(reservedGenerics); + if (candidate.getKey().type instanceof LambdaType) { + if (candidate.getKey().acceptsLambda(arguments.get(argIndex).getSqlLambda(), reservedCopy, allowCasts)) { + final Node node = candidate.getValue(); + getUdfCandidates(arguments, argIndex + 1, node, candidates, reservedCopy, allowCasts); + } + } else { + if (candidate.getKey().accepts(arg, reservedCopy, allowCasts)) { + final Node node = candidate.getValue(); + getUdfCandidates(arguments, argIndex + 1, node, candidates, reservedCopy, allowCasts); + } + } + } + } + private void getCandidates( final List arguments, final int argIndex, @@ -215,6 +275,28 @@ private KsqlException createNoMatchingFunctionException(final List para ); } + private KsqlException createNoMatchingFunctionExceptionSqlArgument(final List paramTypes) { + LOG.debug("Current UdfIndex:\n{}", describe()); + + final String requiredTypes = paramTypes.stream() + .map(type -> type.getSqlType() == null ? "null" : type.getSqlType().toString(FormatOptions.noEscape())) + .collect(Collectors.joining(", ", "(", ")")); + + final String acceptedTypes = allFunctions.values().stream() + .map(UdfIndex::formatAvailableSignatures) + .collect(Collectors.joining(System.lineSeparator())); + + return new KsqlException("Function '" + udfName + + "' does not accept parameters " + requiredTypes + "." + + System.lineSeparator() + + "Valid alternatives are:" + + System.lineSeparator() + + acceptedTypes + + System.lineSeparator() + + "For detailed information on a function run: DESCRIBE FUNCTION ;" + ); + } + public Collection values() { return allFunctions.values(); } @@ -362,6 +444,21 @@ boolean accepts(final SqlType argument, final Map reserved } // CHECKSTYLE_RULES.ON: BooleanExpressionComplexity + // CHECKSTYLE_RULES.OFF: BooleanExpressionComplexity + boolean acceptsLambda(final SqlLambda lambda, final Map reservedGenerics, + final boolean allowCasts) { + if (lambda == null) { + return true; + } + + if (GenericsUtil.hasGenerics(type)) { + return reserveLambdaGenerics(type, lambda, reservedGenerics); + } + + return ParamTypes.isLambdaCompatible(lambda, type); + } + // CHECKSTYLE_RULES.ON: BooleanExpressionComplexity + private static boolean reserveGenerics( final ParamType schema, final SqlType argument, @@ -384,6 +481,28 @@ private static boolean reserveGenerics( return true; } + private static boolean reserveLambdaGenerics( + final ParamType schema, + final SqlLambda argument, + final Map reservedGenerics + ) { + /*if (!GenericsUtil.instanceOf(schema, argument)) { + return false; + }*/ + + final Map genericMapping = GenericsUtil + .resolveLambdaGenerics(schema, argument); + + for (final Entry entry : genericMapping.entrySet()) { + final SqlType old = reservedGenerics.putIfAbsent(entry.getKey(), entry.getValue()); + if (old != null && !old.equals(entry.getValue())) { + return false; + } + } + + return true; + } + @Override public String toString() { return type + (isVararg ? "(VARARG)" : ""); diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/types/LambdaType.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/types/LambdaType.java new file mode 100644 index 000000000000..d13713be912e --- /dev/null +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/types/LambdaType.java @@ -0,0 +1,73 @@ +/* + * 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.types; + +import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.Objects; + +public final class LambdaType extends ObjectType { + + private final ImmutableList inputTypes; + private final ParamType returnType; + + public LambdaType( + final List inputTypes, + final ParamType returnType + ) { + this.inputTypes = ImmutableList.copyOf( + Objects.requireNonNull(inputTypes, "inputTypes")); + this.returnType = Objects.requireNonNull(returnType, "returnType"); + } + + public static LambdaType of( + final List inputTypes, + final ParamType returnType + ) { + return new LambdaType(inputTypes, returnType); + } + + public List inputTypes() { + return inputTypes; + } + + public ParamType returnType() { + return returnType; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final LambdaType lambdaType = (LambdaType) o; + return Objects.equals(inputTypes, lambdaType.inputTypes) + && Objects.equals(returnType, lambdaType.returnType); + } + + @Override + public int hashCode() { + return Objects.hash(inputTypes, returnType); + } + + @Override + public String toString() { + return "LAMBDA<" + inputTypes + ", " + returnType + ">"; + } +} diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/types/ParamTypes.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/types/ParamTypes.java index 0cfb0751ee63..193a66901acb 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/function/types/ParamTypes.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/types/ParamTypes.java @@ -19,6 +19,7 @@ import io.confluent.ksql.schema.ksql.types.SqlArray; import io.confluent.ksql.schema.ksql.types.SqlBaseType; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlMap; import io.confluent.ksql.schema.ksql.types.SqlStruct; import io.confluent.ksql.schema.ksql.types.SqlStruct.Field; @@ -88,6 +89,25 @@ private static boolean isStructCompatible(final SqlType actual, final ParamType return actualStruct.fields().size() == ((StructType) declared).getSchema().size(); } + public static boolean isLambdaCompatible(final SqlLambda actual, final ParamType declared) { + final LambdaType declaredLambda = (LambdaType) declared; + if (actual.getInputType().size() != declaredLambda.inputTypes().size()) { + return false; + } + int i = 0; + for (final ParamType paramType: declaredLambda.inputTypes()) { + if (!areCompatible(actual.getInputType().get(i), paramType)) { + return false; + } + i++; + } + + if (!areCompatible(actual.getReturnType(), declaredLambda.returnType())) { + return false; + } + return true; + } + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private static boolean isPrimitiveMatch( final SqlType actual, diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java index 664dae15ad31..6a6d9f048377 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java @@ -211,6 +211,7 @@ PreparedStatement prepare(final ParsedStatement stmt, final Map visitLambdaExpression( ctx.process(expression.getBody()); ctx.getContext().removeLambdaArgs(); ctx.getContext().addLambdaArgs(previousLambdaArgs); - return visitExpression(expression, ctx); } } - private static class SanitizerContext { + public static class SanitizerContext { final Set lambdaArgs = new HashSet<>(); private void addLambdaArgs(final Set newArguments) { diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/LambdaContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionContext.java similarity index 84% rename from ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/LambdaContext.java rename to ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionContext.java index 8d7e2c5c1a89..53620a47a423 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/LambdaContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionContext.java @@ -17,17 +17,15 @@ import io.confluent.ksql.util.KsqlException; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Objects; -public class LambdaContext { +public class ExpressionContext { private final List lambdaArguments; - public LambdaContext(final List lambdaArguments) { - this.lambdaArguments = new ArrayList<>( - Objects.requireNonNull(lambdaArguments, "lambdaArguments")); + public ExpressionContext(final List lambdaArguments) { + this.lambdaArguments = Objects.requireNonNull(lambdaArguments, "lambdaArguments"); } public void addLambdaArguments(final List newArguments) { diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionTreeRewriter.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionTreeRewriter.java index f6df9db645c9..522c1e30a8ee 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionTreeRewriter.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/rewrite/ExpressionTreeRewriter.java @@ -56,6 +56,8 @@ import io.confluent.ksql.execution.expression.tree.Type; import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.execution.expression.tree.WhenClause; + +import java.util.ArrayList; import java.util.List; import java.util.Map.Entry; import java.util.Objects; @@ -74,7 +76,7 @@ * @param A context type to be passed through to the plugin. */ public final class ExpressionTreeRewriter { - + public static final class Context { private final C context; private final ExpressionVisitor rewriter; diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/FunctionLoaderUtils.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/FunctionLoaderUtils.java index b6dd6c498219..5a41a7974dcd 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/FunctionLoaderUtils.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/FunctionLoaderUtils.java @@ -18,6 +18,7 @@ import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.execution.function.UdfUtil; import io.confluent.ksql.function.types.GenericType; +import io.confluent.ksql.function.types.LambdaType; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.function.types.ParamTypes; import io.confluent.ksql.function.udf.Udf; @@ -25,6 +26,7 @@ import io.confluent.ksql.function.udf.UdfSchemaProvider; import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.schema.ksql.SqlTypeParser; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlException; @@ -32,6 +34,7 @@ import java.lang.reflect.Method; import java.lang.reflect.Parameter; import java.lang.reflect.Type; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -177,14 +180,24 @@ static SchemaProvider handleUdfReturnSchema( final Map genericMapping = new HashMap<>(); for (int i = 0; i < Math.min(parameters.size(), arguments.size()); i++) { final ParamType schema = parameters.get(i); + if (schema instanceof LambdaType) { + List types = new ArrayList<>(); + final int last = arguments.size() - 1; + for (int j = 0; j < last; j++) { + types.add(arguments.get(j)); + } + SqlLambda sqlLambda = SqlLambda.of(types, arguments.get(last)); + genericMapping.putAll(GenericsUtil.resolveLambdaGenerics(schema, sqlLambda)); + } else { + // we resolve any variadic as if it were an array so that the type + // structure matches the input type + final SqlType instance = isVariadic && i == parameters.size() - 1 + ? SqlTypes.array(arguments.get(i)) + : arguments.get(i); - // we resolve any variadic as if it were an array so that the type - // structure matches the input type - final SqlType instance = isVariadic && i == parameters.size() - 1 - ? SqlTypes.array(arguments.get(i)) - : arguments.get(i); + genericMapping.putAll(GenericsUtil.resolveGenerics(schema, instance)); + } - genericMapping.putAll(GenericsUtil.resolveGenerics(schema, instance)); } return GenericsUtil.applyResolved(javaReturnSchema, genericMapping); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/UdafTypes.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/UdafTypes.java index 2f118d2498bb..cccde1e86733 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/UdafTypes.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/UdafTypes.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import io.confluent.ksql.execution.codegen.helpers.TriFunction; import io.confluent.ksql.execution.function.UdfUtil; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.name.FunctionName; @@ -34,6 +35,8 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Function; import org.apache.kafka.connect.data.Struct; class UdafTypes { @@ -53,6 +56,9 @@ class UdafTypes { .add(List.class) .add(Map.class) .add(Timestamp.class) + .add(Function.class) + .add(BiFunction.class) + .add(TriFunction.class) .build(); private final Type inputType; 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 new file mode 100644 index 000000000000..26aaa1537fd1 --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayTransform.java @@ -0,0 +1,55 @@ +/* + * 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/map/MapReduce.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapReduce.java new file mode 100644 index 000000000000..2064dcc98698 --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapReduce.java @@ -0,0 +1,59 @@ +/* + * 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 = "map_reduce", + category = FunctionCategory.MAP, + description = "Reduce the input map down to a single value " + + "using an initial state and a function. " + + "The initial state (s) and 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 MapReduce { + + @Udf + public S mapReduce( + @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/main/java/io/confluent/ksql/function/udf/map/MapTransform.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapTransform.java new file mode 100644 index 000000000000..b321665d57d8 --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/map/MapTransform.java @@ -0,0 +1,59 @@ +/* + * 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.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.function.BiFunction; +import java.util.stream.Collectors; + +/** + * Transform a map's key and values using two lambda functions + */ +@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", + author = KsqlConstants.CONFLUENT_AUTHOR +) +public class MapTransform { + + @Udf + public Map mapTransform( + @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 + ) { + if (map == null) { + return null; + } + + return map.entrySet() + .stream() + .collect(Collectors.toMap( + entry -> biFunction1.apply(entry.getKey(), entry.getValue()), + entry -> biFunction2.apply(entry.getKey(), entry.getValue()))); + } +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/LambdaContextTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/LambdaContextTest.java deleted file mode 100644 index f7bcf1adbddb..000000000000 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/LambdaContextTest.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.engine.rewrite; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThrows; - -import com.google.common.collect.ImmutableList; -import io.confluent.ksql.parser.tree.Statement; -import io.confluent.ksql.util.KsqlException; -import org.junit.Test; - -public class LambdaContextTest { - @Test - public void shouldThrowIfSourceDoesNotExist() { - // Given: - final LambdaContext context = new LambdaContext(ImmutableList.of("X")); - - // When: - context.addLambdaArguments(ImmutableList.of("Z", "Y")); - - // Then: - assertThat(context.getLambdaArguments(), is(ImmutableList.of("X", "Z", "Y"))); - } - - @Test - public void shouldThrowIfLambdaArgumentAlreadyUsed() { - // Given: - final LambdaContext context = new LambdaContext(ImmutableList.of("X")); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> context.addLambdaArguments(ImmutableList.of("X", "Y")) - ); - - // Then: - assertThat(e.getMessage(), containsString( - "Duplicate lambda arguments are not allowed.")); - } -} \ No newline at end of file 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 new file mode 100644 index 000000000000..adeda725aadb --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayTransformTest.java @@ -0,0 +1,83 @@ +/* + * 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/map/MapReduceTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapReduceTest.java new file mode 100644 index 000000000000..fb931fd0dd40 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapReduceTest.java @@ -0,0 +1,77 @@ +/* + * 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 MapReduceTest { + + private MapReduce udf; + + @Before + public void setUp() { + udf = new MapReduce(); + } + + @Test + public void shouldReturnNullForNullMap() { + assertThat(udf.mapReduce(null, 0, triFunction1()), is(nullValue())); + } + + @Test + public void shouldReduceMap() { + final Map map1 = new HashMap<>(); + assertThat(udf.mapReduce(map1, 3, triFunction1()), is(3)); + map1.put(4, 3); + map1.put(6, 2); + assertThat(udf.mapReduce(map1, 42, triFunction1()), is(57)); + assertThat(udf.mapReduce(map1, -4, triFunction1()), is(11)); + map1.put(0,0); + assertThat(udf.mapReduce(map1, 0, triFunction1()), is(15)); + + final Map map2 = new HashMap<>(); + assertThat(udf.mapReduce(map2, "", triFunction2()), is("")); + map2.put("a", 42); + map2.put("b", 11); + assertThat(udf.mapReduce(map2, "", triFunction2()), is("ba")); + assertThat(udf.mapReduce(map2, "string", triFunction2()), is("bastring")); + map2.put("c",0); + map2.put("d",15); + map2.put("e",-5); + assertThat(udf.mapReduce(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-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 new file mode 100644 index 000000000000..2f8cea314ac6 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/map/MapTransformTest.java @@ -0,0 +1,81 @@ +/* + * 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-execution/src/main/java/io/confluent/ksql/execution/codegen/CodeGenRunner.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/CodeGenRunner.java index d4bcc8b8b0d7..27d4a918ed2e 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/CodeGenRunner.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/CodeGenRunner.java @@ -39,14 +39,21 @@ import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.schema.ksql.SchemaConverters.SqlToJavaTypeConverter; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlArray; +import io.confluent.ksql.schema.ksql.types.SqlLambda; +import io.confluent.ksql.schema.ksql.types.SqlMap; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.kafka.connect.data.Schema; import org.codehaus.commons.compiler.CompileException; import org.codehaus.commons.compiler.CompilerFactoryFactory; @@ -108,8 +115,8 @@ public CodeGenRunner( public CodeGenSpec getCodeGenSpec(final Expression expression) { final Visitor visitor = new Visitor(); - - visitor.process(expression, null); + final CodeGenTypeContext context = new CodeGenTypeContext(); + visitor.process(expression, context); return visitor.spec.build(); } @@ -139,9 +146,11 @@ public ExpressionMetadata buildCodeGenFromParseTree( return new ExpressionMetadata(ee, spec, returnType, expression); } catch (KsqlException | CompileException e) { + e.printStackTrace(); throw new KsqlException("Invalid " + type + ": " + e.getMessage() + ". expression:" + expression + ", schema:" + schema, e); } catch (final Exception e) { + e.printStackTrace(); throw new RuntimeException("Unexpected error generating code for " + type + ". expression:" + expression, e); } @@ -164,7 +173,34 @@ public static IExpressionEvaluator cook( return ee; } - private final class Visitor extends TraversalExpressionVisitor { + private static class CodeGenTypeContext { + + private List inputTypes = new ArrayList<>(); + private Map lambdaTypeMapping = new HashMap<>(); + + List getInputTypes() { + if (inputTypes.size() == 0) { + return null; + } + return inputTypes; + } + + void addInputType(final SqlType inputType) { + this.inputTypes.add(inputType); + } + + void mapInputTypes(final List argumentList){ + for (int i = 0; i < argumentList.size(); i++) { + this.lambdaTypeMapping.putIfAbsent(argumentList.get(i), inputTypes.get(i)); + } + } + + Map getLambdaTypeMapping(){ + return this.lambdaTypeMapping; + } + } + + private final class Visitor extends TraversalExpressionVisitor { private final CodeGenSpec.Builder spec; @@ -173,23 +209,44 @@ private Visitor() { } @Override - public Void visitLikePredicate(final LikePredicate node, final Void context) { - process(node.getValue(), null); - process(node.getPattern(), null); + public Void visitLikePredicate(final LikePredicate node, final CodeGenTypeContext context) { + process(node.getValue(), context); + process(node.getPattern(), context); return null; } @Override - public Void visitFunctionCall(final FunctionCall node, final Void context) { - final List argumentTypes = new ArrayList<>(); + public Void visitFunctionCall(final FunctionCall node, final CodeGenTypeContext context) { + final List newArgumentTypes = new ArrayList<>(); final FunctionName functionName = node.getName(); + final UdfFactory holder = functionRegistry.getUdfFactory(functionName); for (final Expression argExpr : node.getArguments()) { - process(argExpr, null); - argumentTypes.add(expressionTypeManager.getExpressionSqlType(argExpr)); + process(argExpr, context); + SqlType newSqlType = expressionTypeManager.getExpressionSqlType(argExpr, context.getLambdaTypeMapping(), context.getInputTypes()); + // for lambdas - if we see this it's the array/map being passed in. We save the type for later + + if (shouldSetInputType(node, context)) { + if (newSqlType instanceof SqlArray) { + SqlArray inputArray = (SqlArray) newSqlType; + context.addInputType(inputArray.getItemType()); + } else if (newSqlType instanceof SqlMap) { + SqlMap inputMap = (SqlMap) newSqlType; + context.addInputType(inputMap.getKeyType()); + context.addInputType(inputMap.getValueType()); + } else { + context.addInputType(newSqlType); + } + } + + if (argExpr instanceof LambdaFunctionCall) { + newArgumentTypes.add(new SqlArgument(null, SqlLambda.of(context.getInputTypes(), newSqlType))); + + } else { + newArgumentTypes.add(new SqlArgument(newSqlType, null)); + } } - final UdfFactory holder = functionRegistry.getUdfFactory(functionName); - final KsqlScalarFunction function = holder.getFunction(argumentTypes); + final KsqlScalarFunction function = holder.getUdfFunction(newArgumentTypes); spec.addFunction( function.name(), function.newInstance(ksqlConfig) @@ -199,7 +256,7 @@ public Void visitFunctionCall(final FunctionCall node, final Void context) { } @Override - public Void visitSubscriptExpression(final SubscriptExpression node, final Void context) { + public Void visitSubscriptExpression(final SubscriptExpression node, final CodeGenTypeContext context) { if (node.getBase() instanceof UnqualifiedColumnReferenceExp) { final UnqualifiedColumnReferenceExp arrayBaseName = (UnqualifiedColumnReferenceExp) node.getBase(); @@ -212,13 +269,13 @@ public Void visitSubscriptExpression(final SubscriptExpression node, final Void } @Override - public Void visitCreateArrayExpression(final CreateArrayExpression exp, final Void context) { + public Void visitCreateArrayExpression(final CreateArrayExpression exp, final CodeGenTypeContext context) { exp.getValues().forEach(val -> process(val, context)); return null; } @Override - public Void visitCreateMapExpression(final CreateMapExpression exp, final Void context) { + public Void visitCreateMapExpression(final CreateMapExpression exp, final CodeGenTypeContext context) { for (Entry entry : exp.getMap().entrySet()) { process(entry.getKey(), context); process(entry.getValue(), context); @@ -229,7 +286,7 @@ public Void visitCreateMapExpression(final CreateMapExpression exp, final Void c @Override public Void visitStructExpression( final CreateStructExpression exp, - final Void context + final CodeGenTypeContext context ) { exp.getFields().forEach(val -> process(val.getValue(), context)); final Schema schema = SchemaConverters @@ -243,21 +300,22 @@ public Void visitStructExpression( @Override public Void visitUnqualifiedColumnReference( final UnqualifiedColumnReferenceExp node, - final Void context + final CodeGenTypeContext context ) { addRequiredColumn(node.getColumnName()); return null; } @Override - public Void visitDereferenceExpression(final DereferenceExpression node, final Void context) { - process(node.getBase(), null); + public Void visitDereferenceExpression(final DereferenceExpression node, final CodeGenTypeContext context) { + process(node.getBase(), context); return null; } @Override - public Void visitLambdaExpression(final LambdaFunctionCall node, final Void context) { - process(node.getBody(), null); + public Void visitLambdaExpression(final LambdaFunctionCall node, final CodeGenTypeContext context) { + context.mapInputTypes(node.getArguments()); + process(node.getBody(), context); return null; } @@ -274,5 +332,11 @@ private void addRequiredColumn(final ColumnName columnName) { column.index() ); } + + private Boolean shouldSetInputType(final FunctionCall node, final CodeGenTypeContext context) { + return (context.getInputTypes() == null) + || (node.getName().equals(FunctionName.of("REDUCE_MAP")) && context.getInputTypes().size() == 2) + || (node.getName().equals(FunctionName.of("REDUCE_ARRAY")) && context.getInputTypes().size() == 1); + } } } diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitor.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitor.java index ff11c04ea1ab..5a293706090a 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitor.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitor.java @@ -29,6 +29,7 @@ import io.confluent.ksql.execution.codegen.helpers.ArrayBuilder; import io.confluent.ksql.execution.codegen.helpers.CastEvaluator; import io.confluent.ksql.execution.codegen.helpers.InListEvaluator; +import io.confluent.ksql.execution.codegen.helpers.LambdaUtil; import io.confluent.ksql.execution.codegen.helpers.LikeEvaluator; import io.confluent.ksql.execution.codegen.helpers.MapBuilder; import io.confluent.ksql.execution.codegen.helpers.NullSafe; @@ -76,6 +77,7 @@ import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.GenericsUtil; import io.confluent.ksql.function.KsqlFunction; +import io.confluent.ksql.function.TriFunction; import io.confluent.ksql.function.UdfFactory; import io.confluent.ksql.function.types.ArrayType; import io.confluent.ksql.function.types.ParamType; @@ -86,12 +88,14 @@ import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.SchemaConverters; +import io.confluent.ksql.schema.ksql.SqlArgument; import io.confluent.ksql.schema.ksql.SqlBooleans; import io.confluent.ksql.schema.ksql.SqlDoubles; import io.confluent.ksql.schema.ksql.SqlTimestamps; import io.confluent.ksql.schema.ksql.types.SqlArray; import io.confluent.ksql.schema.ksql.types.SqlBaseType; import io.confluent.ksql.schema.ksql.types.SqlDecimal; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlMap; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; @@ -102,6 +106,8 @@ import java.math.BigDecimal; import java.math.MathContext; import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -121,7 +127,6 @@ public class SqlToJavaVisitor { public static final List JAVA_IMPORTS = ImmutableList.of( "io.confluent.ksql.execution.codegen.helpers.ArrayAccess", "io.confluent.ksql.execution.codegen.helpers.SearchedCaseFunction", - "io.confluent.ksql.execution.codegen.helpers.TriFunction", "io.confluent.ksql.execution.codegen.helpers.SearchedCaseFunction.LazyWhenClause", "java.sql.Timestamp", "java.util.Arrays", @@ -135,6 +140,7 @@ public class SqlToJavaVisitor { "java.util.function.Supplier", Function.class.getCanonicalName(), BiFunction.class.getCanonicalName(), + TriFunction.class.getCanonicalName(), DecimalUtil.class.getCanonicalName(), BigDecimal.class.getCanonicalName(), MathContext.class.getCanonicalName(), @@ -224,12 +230,44 @@ public String process(final Expression expression) { } private String formatExpression(final Expression expression) { + final SqlToJavaTypeContext context = new SqlToJavaTypeContext(); final Pair expressionFormatterResult = - new Formatter(functionRegistry).process(expression, null); + new Formatter(functionRegistry).process(expression, context); return expressionFormatterResult.getLeft(); } - private class Formatter implements ExpressionVisitor, Void> { + private static class SqlToJavaTypeContext { + + private List inputTypes = new ArrayList<>(); + private Map lambdaTypeMapping = new HashMap<>(); + + List getInputTypes() { + if (inputTypes.size() == 0) { + return null; + } + return inputTypes; + } + + void addInputType(final SqlType inputType) { + this.inputTypes.add(inputType); + } + + void mapInputTypes(final List argumentList){ + for (int i = 0; i < argumentList.size(); i++) { + this.lambdaTypeMapping.putIfAbsent(argumentList.get(i), inputTypes.get(i)); + } + } + + SqlType getLambdaType(final String name) { + return lambdaTypeMapping.get(name); + } + + Map getLambdaTypeMapping() { + return this.lambdaTypeMapping; + } + } + + private class Formatter implements ExpressionVisitor, SqlToJavaTypeContext> { private final FunctionRegistry functionRegistry; @@ -253,19 +291,21 @@ private Pair visitUnsupported(final Expression expression) { } @Override - public Pair visitType(final Type node, final Void context) { + public Pair visitType(final Type node, final SqlToJavaTypeContext context) { return visitIllegalState(node); } @Override - public Pair visitWhenClause(final WhenClause whenClause, final Void context) { + public Pair visitWhenClause( + final WhenClause whenClause, final SqlToJavaTypeContext context + ) { return visitIllegalState(whenClause); } @Override public Pair visitInPredicate( final InPredicate inPredicate, - final Void context + final SqlToJavaTypeContext context ) { final InPredicate preprocessed = InListEvaluator .preprocess(inPredicate, expressionTypeManager); @@ -285,14 +325,14 @@ public Pair visitInPredicate( @Override public Pair visitInListExpression( - final InListExpression inListExpression, final Void context + final InListExpression inListExpression, final SqlToJavaTypeContext context ) { return visitUnsupported(inListExpression); } @Override public Pair visitTimestampLiteral( - final TimestampLiteral node, final Void context + final TimestampLiteral node, final SqlToJavaTypeContext context ) { return new Pair<>(node.toString(), SqlTypes.TIMESTAMP); } @@ -300,14 +340,14 @@ public Pair visitTimestampLiteral( @Override public Pair visitTimeLiteral( final TimeLiteral timeLiteral, - final Void context + final SqlToJavaTypeContext context ) { return visitUnsupported(timeLiteral); } @Override public Pair visitSimpleCaseExpression( - final SimpleCaseExpression simpleCaseExpression, final Void context + final SimpleCaseExpression simpleCaseExpression, final SqlToJavaTypeContext context ) { return visitUnsupported(simpleCaseExpression); } @@ -315,13 +355,15 @@ public Pair visitSimpleCaseExpression( @Override public Pair visitBooleanLiteral( final BooleanLiteral node, - final Void context + final SqlToJavaTypeContext context ) { return new Pair<>(String.valueOf(node.getValue()), SqlTypes.BOOLEAN); } @Override - public Pair visitStringLiteral(final StringLiteral node, final Void context) { + public Pair visitStringLiteral( + final StringLiteral node, final SqlToJavaTypeContext context + ) { return new Pair<>( "\"" + StringEscapeUtils.escapeJava(node.getValue()) + "\"", SqlTypes.STRING @@ -329,14 +371,16 @@ public Pair visitStringLiteral(final StringLiteral node, final } @Override - public Pair visitDoubleLiteral(final DoubleLiteral node, final Void context) { + public Pair visitDoubleLiteral( + final DoubleLiteral node, final SqlToJavaTypeContext context + ) { return new Pair<>(node.toString(), SqlTypes.DOUBLE); } @Override public Pair visitDecimalLiteral( final DecimalLiteral decimalLiteral, - final Void context + final SqlToJavaTypeContext context ) { return new Pair<>( "new BigDecimal(\"" + decimalLiteral.getValue() + "\")", @@ -345,26 +389,40 @@ public Pair visitDecimalLiteral( } @Override - public Pair visitNullLiteral(final NullLiteral node, final Void context) { + public Pair visitNullLiteral( + final NullLiteral node, final SqlToJavaTypeContext context + ) { return new Pair<>("null", null); } @Override + // CHECKSTYLE_RULES.OFF: TodoComment public Pair visitLambdaExpression( - final LambdaFunctionCall lambdaFunctionCall, final Void context) { - return visitUnsupported(lambdaFunctionCall); + final LambdaFunctionCall exp, final SqlToJavaTypeContext context) { + + context.mapInputTypes(exp.getArguments()); + + final Pair lambdaBody = process(exp.getBody(), context); + + final List>> argPairs = new ArrayList<>(); + + for (final String lambdaArg: exp.getArguments()) { + argPairs.add(new Pair<>(lambdaArg, SchemaConverters.sqlToJavaConverter().toJavaType(context.getLambdaType(lambdaArg)))); + } + return new Pair<>(LambdaUtil.function(argPairs, lambdaBody.getLeft()), + expressionTypeManager.getExpressionSqlType(exp, context.getLambdaTypeMapping(), context.getInputTypes())); } @Override public Pair visitLambdaVariable( - final LambdaVariable lambdaVariable, final Void context) { - return visitUnsupported(lambdaVariable); + final LambdaVariable lambdaVariable, final SqlToJavaTypeContext context) { + return new Pair<>(lambdaVariable.getValue(), context.getLambdaType(lambdaVariable.getValue())); } @Override public Pair visitUnqualifiedColumnReference( final UnqualifiedColumnReferenceExp node, - final Void context + final SqlToJavaTypeContext context ) { final ColumnName fieldName = node.getColumnName(); final Column schemaColumn = schema.findValueColumn(node.getColumnName()) @@ -377,7 +435,7 @@ public Pair visitUnqualifiedColumnReference( @Override public Pair visitQualifiedColumnReference( final QualifiedColumnReferenceExp node, - final Void context + final SqlToJavaTypeContext context ) { throw new UnsupportedOperationException( "Qualified column reference must be resolved to unqualified reference before codegen" @@ -386,7 +444,7 @@ public Pair visitQualifiedColumnReference( @Override public Pair visitDereferenceExpression( - final DereferenceExpression node, final Void context + final DereferenceExpression node, final SqlToJavaTypeContext context ) { final SqlType functionReturnSchema = expressionTypeManager.getExpressionSqlType(node); final String javaReturnType = @@ -400,30 +458,55 @@ public Pair visitDereferenceExpression( return new Pair<>(codeString, functionReturnSchema); } - public Pair visitLongLiteral(final LongLiteral node, final Void context) { + public Pair visitLongLiteral( + final LongLiteral node, final SqlToJavaTypeContext context + ) { return new Pair<>(node.getValue() + "L", SqlTypes.BIGINT); } @Override public Pair visitIntegerLiteral( final IntegerLiteral node, - final Void context + final SqlToJavaTypeContext context ) { return new Pair<>(Integer.toString(node.getValue()), SqlTypes.INTEGER); } @Override - public Pair visitFunctionCall(final FunctionCall node, final Void context) { + public Pair visitFunctionCall( + final FunctionCall node, final SqlToJavaTypeContext context + ) { final FunctionName functionName = node.getName(); final String instanceName = funNameToCodeName.apply(functionName); final UdfFactory udfFactory = functionRegistry.getUdfFactory(node.getName()); - final List argumentSchemas = node.getArguments().stream() - .map(expressionTypeManager::getExpressionSqlType) - .collect(Collectors.toList()); + final List argumentSchemas = new ArrayList<>(); + final List newArgumentSchemas = new ArrayList<>(); + for (final Expression argExpr : node.getArguments()) { + SqlType newSqlType = expressionTypeManager.getExpressionSqlType(argExpr, context.getLambdaTypeMapping(), context.getInputTypes()); + // for lambdas: if it's the array/map being passed in we save the type for later + if (shouldSetInputType(node, context)) { + if (newSqlType instanceof SqlArray) { + SqlArray inputArray = (SqlArray) newSqlType; + context.addInputType(inputArray.getItemType()); + } else if (newSqlType instanceof SqlMap) { + SqlMap inputMap = (SqlMap) newSqlType; + context.addInputType(inputMap.getKeyType()); + context.addInputType(inputMap.getValueType()); + } else { + context.addInputType(newSqlType); + } + } + if (argExpr instanceof LambdaFunctionCall) { + newArgumentSchemas.add(new SqlArgument(null, SqlLambda.of(context.getInputTypes(), newSqlType))); + } else { + newArgumentSchemas.add(new SqlArgument(newSqlType, null)); + } + argumentSchemas.add(newSqlType); + } - final KsqlFunction function = udfFactory.getFunction(argumentSchemas); + final KsqlFunction function = udfFactory.getUdfFunction(newArgumentSchemas); final SqlType functionReturnSchema = function.getReturnType(argumentSchemas); final String javaReturnType = @@ -432,6 +515,7 @@ public Pair visitFunctionCall(final FunctionCall node, final Vo final List arguments = node.getArguments(); final StringJoiner joiner = new StringJoiner(", "); + for (int i = 0; i < arguments.size(); i++) { final Expression arg = arguments.get(i); final SqlType sqlType = argumentSchemas.get(i); @@ -443,7 +527,9 @@ public Pair visitFunctionCall(final FunctionCall node, final Vo paramType = function.parameters().get(i); } - joiner.add(process(convertArgument(arg, sqlType, paramType), context).getLeft()); + final Pair pair = + process(convertArgument(arg, sqlType, paramType), context); + joiner.add(pair.getLeft()); } @@ -472,7 +558,7 @@ private Expression convertArgument( @Override public Pair visitLogicalBinaryExpression( - final LogicalBinaryExpression node, final Void context + final LogicalBinaryExpression node, final SqlToJavaTypeContext context ) { if (node.getType() == LogicalBinaryExpression.Type.OR) { return new Pair<>( @@ -493,7 +579,9 @@ public Pair visitLogicalBinaryExpression( } @Override - public Pair visitNotExpression(final NotExpression node, final Void context) { + public Pair visitNotExpression( + final NotExpression node, final SqlToJavaTypeContext context + ) { final String exprString = process(node.getValue(), context).getLeft(); return new Pair<>("(!" + exprString + ")", SqlTypes.BOOLEAN); } @@ -646,7 +734,7 @@ private String toTimestamp(final SqlType schema, final int index) { @Override public Pair visitComparisonExpression( - final ComparisonExpression node, final Void context + final ComparisonExpression node, final SqlToJavaTypeContext context ) { final Pair left = process(node.getLeft(), context); final Pair right = process(node.getRight(), context); @@ -688,16 +776,17 @@ public Pair visitComparisonExpression( } @Override - public Pair visitCast(final Cast node, final Void context) { + public Pair visitCast(final Cast node, final SqlToJavaTypeContext context) { final Pair expr = process(node.getExpression(), context); final SqlType to = node.getType().getSqlType(); + //final Pair pair = Pair.of(genCastCode(expr, to), to); return Pair.of(genCastCode(expr, to), to); } @Override public Pair visitIsNullPredicate( final IsNullPredicate node, - final Void context + final SqlToJavaTypeContext context ) { final Pair value = process(node.getValue(), context); return new Pair<>("((" + value.getLeft() + ") == null )", SqlTypes.BOOLEAN); @@ -706,7 +795,7 @@ public Pair visitIsNullPredicate( @Override public Pair visitIsNotNullPredicate( final IsNotNullPredicate node, - final Void context + final SqlToJavaTypeContext context ) { final Pair value = process(node.getValue(), context); return new Pair<>("((" + value.getLeft() + ") != null )", SqlTypes.BOOLEAN); @@ -714,7 +803,7 @@ public Pair visitIsNotNullPredicate( @Override public Pair visitArithmeticUnary( - final ArithmeticUnaryExpression node, final Void context + final ArithmeticUnaryExpression node, final SqlToJavaTypeContext context ) { final Pair value = process(node.getValue(), context); switch (node.getSign()) { @@ -761,12 +850,13 @@ private Pair visitArithmeticPlus(final Pair va @Override public Pair visitArithmeticBinary( - final ArithmeticBinaryExpression node, final Void context + final ArithmeticBinaryExpression node, final SqlToJavaTypeContext context ) { final Pair left = process(node.getLeft(), context); final Pair right = process(node.getRight(), context); - final SqlType schema = expressionTypeManager.getExpressionSqlType(node); + final SqlType schema = + expressionTypeManager.getExpressionSqlType(node, context.getLambdaTypeMapping(), context.getInputTypes()); if (schema.baseType() == SqlBaseType.DECIMAL) { final SqlDecimal decimal = (SqlDecimal) schema; @@ -808,7 +898,7 @@ public Pair visitArithmeticBinary( @Override public Pair visitSearchedCaseExpression( - final SearchedCaseExpression node, final Void context + final SearchedCaseExpression node, final SqlToJavaTypeContext context ) { final String functionClassName = SearchedCaseFunction.class.getSimpleName(); final List whenClauses = node @@ -820,7 +910,8 @@ public Pair visitSearchedCaseExpression( )) .collect(Collectors.toList()); - final SqlType resultSchema = expressionTypeManager.getExpressionSqlType(node); + final SqlType resultSchema = + expressionTypeManager.getExpressionSqlType(node, context.getLambdaTypeMapping(), context.getInputTypes()); final String resultSchemaString = SchemaConverters.sqlToJavaConverter().toJavaType(resultSchema).getCanonicalName(); @@ -857,7 +948,9 @@ private String buildSupplierCode(final String typeString, final String code) { } @Override - public Pair visitLikePredicate(final LikePredicate node, final Void context) { + public Pair visitLikePredicate( + final LikePredicate node, final SqlToJavaTypeContext context + ) { final String patternString = process(node.getPattern(), context).getLeft(); final String valueString = process(node.getValue(), context).getLeft(); @@ -881,7 +974,7 @@ public Pair visitLikePredicate(final LikePredicate node, final @Override public Pair visitSubscriptExpression( final SubscriptExpression node, - final Void context + final SqlToJavaTypeContext context ) { final SqlType internalSchema = expressionTypeManager.getExpressionSqlType(node.getBase()); @@ -925,7 +1018,7 @@ public Pair visitSubscriptExpression( @Override public Pair visitCreateArrayExpression( final CreateArrayExpression exp, - final Void context + final SqlToJavaTypeContext context ) { final List expressions = CoercionUtil .coerceUserList(exp.getValues(), expressionTypeManager) @@ -948,7 +1041,7 @@ public Pair visitCreateArrayExpression( @Override public Pair visitCreateMapExpression( final CreateMapExpression exp, - final Void context + final SqlToJavaTypeContext context ) { final ImmutableMap map = exp.getMap(); final List keys = CoercionUtil @@ -974,7 +1067,7 @@ public Pair visitCreateMapExpression( @Override public Pair visitStructExpression( final CreateStructExpression node, - final Void context + final SqlToJavaTypeContext context ) { final String schemaName = structToCodeName.apply(node); final StringBuilder struct = new StringBuilder("new Struct(").append(schemaName).append(")"); @@ -996,7 +1089,7 @@ public Pair visitStructExpression( @Override public Pair visitBetweenPredicate( final BetweenPredicate node, - final Void context + final SqlToJavaTypeContext context ) { final Pair compareMin = process( new ComparisonExpression( @@ -1020,7 +1113,10 @@ public Pair visitBetweenPredicate( } private String formatBinaryExpression( - final String operator, final Expression left, final Expression right, final Void context + final String operator, + final Expression left, + final Expression right, + final SqlToJavaTypeContext context ) { return "(" + process(left, context).getLeft() + " " + operator + " " + process(right, context).getLeft() + ")"; @@ -1032,6 +1128,12 @@ private String genCastCode( ) { return CastEvaluator.generateCode(exp.left, exp.right, sqlType, ksqlConfig); } + + private Boolean shouldSetInputType(final FunctionCall node, final SqlToJavaTypeContext context) { + return (context.getInputTypes() == null) + || (node.getName().equals(FunctionName.of("REDUCE_MAP")) && context.getInputTypes().size() == 2) + || (node.getName().equals(FunctionName.of("REDUCE_ARRAY")) && context.getInputTypes().size() == 1); + } } private static final class CaseWhenProcessed { diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtil.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtil.java index 9e86988fad01..7a9deac3f7db 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtil.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtil.java @@ -15,6 +15,7 @@ package io.confluent.ksql.execution.codegen.helpers; +import com.google.common.collect.ImmutableList; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.Pair; @@ -28,7 +29,7 @@ public final class LambdaUtil { private LambdaUtil() { } - + /** * Generate code to build a {@link java.util.function.Function}. * @@ -42,15 +43,7 @@ public static String function( final Class argType, final String lambdaBody ) { - final String javaType = argType.getSimpleName(); - final String function = "new Function() {\n" - + " @Override\n" - + " public Object apply(Object arg) {\n" - + " " + javaType + " " + argName + " = (" + javaType + ") arg;\n" - + " return " + lambdaBody + ";\n" - + " }\n" - + "}"; - return function; + return function(ImmutableList.of(new Pair<>(argName, argType)), lambdaBody); } /** @@ -72,13 +65,14 @@ public static String function( i++; final String javaType = argPair.right.getSimpleName(); arguments.append( - " " + javaType + " " + argPair.left + " = (" + javaType + ") arg" + i + ";\n"); + " " + "final" + " " + javaType + " " + argPair.left + + " = (" + javaType + ") arg" + i + ";\n"); } String functionType; String functionApply; if (argList.size() == 1) { functionType = "Function()"; - functionApply = " public Object apply(Object arg) {\n"; + functionApply = " public Object apply(Object arg1) {\n"; } else if (argList.size() == 2) { functionType = "BiFunction()"; functionApply = " public Object apply(Object arg1, Object arg2) {\n"; @@ -88,7 +82,7 @@ public static String function( } else { throw new KsqlException("Unsupported number of lambda arguments."); } - + final String function = "new " + functionType + " {\n" + " @Override\n" + functionApply diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/TriFunction.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/TriFunction.java index ce4848a6eaad..31139c88fddc 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/TriFunction.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/codegen/helpers/TriFunction.java @@ -28,4 +28,4 @@ default TriFunction andThen( Objects.requireNonNull(after); return (A a, B b, C c) -> after.apply(apply(a, b, c)); } -} +} \ No newline at end of file diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/function/UdfUtil.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/function/UdfUtil.java index e597c19f6486..6ce98efa5638 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/function/UdfUtil.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/function/UdfUtil.java @@ -16,9 +16,11 @@ package io.confluent.ksql.execution.function; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.execution.codegen.helpers.TriFunction; import io.confluent.ksql.function.KsqlFunctionException; import io.confluent.ksql.function.types.ArrayType; import io.confluent.ksql.function.types.GenericType; +import io.confluent.ksql.function.types.LambdaType; import io.confluent.ksql.function.types.MapType; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.function.types.ParamTypes; @@ -31,9 +33,13 @@ import java.lang.reflect.TypeVariable; import java.math.BigDecimal; import java.sql.Timestamp; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; import org.apache.kafka.connect.data.Struct; +import sun.reflect.generics.reflectiveObjects.ParameterizedTypeImpl; public final class UdfUtil { @@ -107,7 +113,9 @@ public static ParamType getSchemaFromType(final Type type) { return schema; } + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private static ParamType handleParameterizedType(final Type type) { + // CHECKSTYLE_RULES.ON: CyclomaticComplexity if (type instanceof ParameterizedType) { final ParameterizedType parameterizedType = (ParameterizedType) type; if (parameterizedType.getRawType() == Map.class) { @@ -128,7 +136,13 @@ private static ParamType handleParameterizedType(final Type type) { // schema annotation if a struct is being used return StructType.ANY_STRUCT; } - + if (type instanceof ParameterizedTypeImpl) { + if (Function.class.isAssignableFrom(((ParameterizedTypeImpl) type).getRawType()) + || BiFunction.class.isAssignableFrom(((ParameterizedTypeImpl) type).getRawType()) + || TriFunction.class.isAssignableFrom(((ParameterizedTypeImpl) type).getRawType())) { + return handleLambdaType((ParameterizedTypeImpl) type); + } + } throw new KsqlException("Type inference is not supported for: " + type); } @@ -154,4 +168,23 @@ private static ParamType handleListType(final ParameterizedType type) { return ArrayType.of(elementParamType); } + + private static ParamType handleLambdaType(final ParameterizedTypeImpl type) { + final List inputParamTypes = new ArrayList<>(); + for (int i = 0; i < type.getActualTypeArguments().length - 1; i++) { + final Type inputType = type.getActualTypeArguments()[i]; + final ParamType inputParamType = inputType instanceof TypeVariable + ? GenericType.of(((TypeVariable) inputType).getName()) + : getSchemaFromType(inputType); + inputParamTypes.add(inputParamType); + } + + final Type returnType = + type.getActualTypeArguments()[type.getActualTypeArguments().length - 1]; + final ParamType returnParamType = returnType instanceof TypeVariable + ? GenericType.of(((TypeVariable) returnType).getName()) + : getSchemaFromType(returnType); + + return LambdaType.of(inputParamTypes, returnParamType); + } } 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 ab6f510feea6..858dea1514e8 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 @@ -60,10 +60,13 @@ import io.confluent.ksql.function.KsqlAggregateFunction; import io.confluent.ksql.function.KsqlTableFunction; import io.confluent.ksql.function.UdfFactory; +import io.confluent.ksql.name.FunctionName; import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.SqlArgument; import io.confluent.ksql.schema.ksql.types.SqlArray; import io.confluent.ksql.schema.ksql.types.SqlBaseType; +import io.confluent.ksql.schema.ksql.types.SqlLambda; import io.confluent.ksql.schema.ksql.types.SqlMap; import io.confluent.ksql.schema.ksql.types.SqlStruct; import io.confluent.ksql.schema.ksql.types.SqlStruct.Builder; @@ -74,8 +77,11 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.VisitorUtil; + import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -96,12 +102,26 @@ public ExpressionTypeManager( public SqlType getExpressionSqlType(final Expression expression) { final ExpressionTypeContext expressionTypeContext = new ExpressionTypeContext(); new Visitor().process(expression, expressionTypeContext); - return expressionTypeContext.getSqlType(); + final SqlType newSqlType = expressionTypeContext.getSqlType(); + return newSqlType; + } + + public SqlType getExpressionSqlType( + final Expression expression, final Map inputMapping, final List inputTypes + ) { + final ExpressionTypeContext expressionTypeContext = new ExpressionTypeContext(); + expressionTypeContext.setLambdaTypes(inputMapping); + expressionTypeContext.setInputTypes(inputTypes); + new Visitor().process(expression, expressionTypeContext); + final SqlType newSqlType = expressionTypeContext.getSqlType(); + return newSqlType; } private static class ExpressionTypeContext { private SqlType sqlType; + private List inputTypes = new ArrayList<>(); + private Map lambdaTypeMapping = new HashMap<>(); SqlType getSqlType() { return sqlType; @@ -110,6 +130,35 @@ SqlType getSqlType() { void setSqlType(final SqlType sqlType) { this.sqlType = sqlType; } + + List getInputTypes() { + if (inputTypes.size() == 0) { + return null; + } + return inputTypes; + } + + void addInputType(final SqlType inputType) { + this.inputTypes.add(inputType); + } + void setInputTypes(final List inputTypes) { + if (inputTypes != null) { + this.inputTypes = inputTypes; + } + } + void mapInputTypes(final List argumentList){ + for (int i = 0; i < inputTypes.size(); i++) { + this.lambdaTypeMapping.putIfAbsent(argumentList.get(i), inputTypes.get(i)); + } + } + + void setLambdaTypes(Map mappings){ + this.lambdaTypeMapping = mappings; + } + + SqlType getLambdaType(final String name) { + return lambdaTypeMapping.get(name); + } } private class Visitor implements ExpressionVisitor { @@ -119,11 +168,17 @@ public Void visitArithmeticBinary( final ArithmeticBinaryExpression node, final ExpressionTypeContext expressionTypeContext ) { process(node.getLeft(), expressionTypeContext); - final SqlType leftType = expressionTypeContext.getSqlType(); + SqlType leftType = expressionTypeContext.getSqlType(); process(node.getRight(), expressionTypeContext); - final SqlType rightType = expressionTypeContext.getSqlType(); - + SqlType rightType = expressionTypeContext.getSqlType(); + /*final SqlType inputType = expressionTypeContext.getInputType(); + if (rightType == SqlTypes.LAMBDALITERAL) { + rightType = inputType == null ? SqlTypes.LAMBDALITERAL : inputType; + } + if (leftType == SqlTypes.LAMBDALITERAL) { + leftType = inputType == null ? SqlTypes.LAMBDALITERAL : inputType; + }*/ final SqlType resultType = node.getOperator().resultType(leftType, rightType); expressionTypeContext.setSqlType(resultType); @@ -143,9 +198,8 @@ public Void visitArithmeticUnary( public Void visitLambdaExpression( final LambdaFunctionCall node, final ExpressionTypeContext context ) { + context.mapInputTypes(node.getArguments()); process(node.getBody(), context); - // TODO: add proper type inference - context.setSqlType(SqlTypes.INTEGER); return null; } @@ -154,8 +208,7 @@ public Void visitLambdaExpression( public Void visitLambdaVariable( final LambdaVariable node, final ExpressionTypeContext expressionTypeContext ) { - // TODO: add proper type inference - expressionTypeContext.setSqlType(SqlTypes.INTEGER); + expressionTypeContext.setSqlType(expressionTypeContext.getLambdaType(node.getValue())); return null; } @@ -178,9 +231,11 @@ public Void visitComparisonExpression( final ComparisonExpression node, final ExpressionTypeContext expressionTypeContext ) { process(node.getLeft(), expressionTypeContext); - final SqlType leftSchema = expressionTypeContext.getSqlType(); + SqlType leftSchema = expressionTypeContext.getSqlType(); + process(node.getRight(), expressionTypeContext); - final SqlType rightSchema = expressionTypeContext.getSqlType(); + SqlType rightSchema = expressionTypeContext.getSqlType(); + if (!ComparisonUtil.isValidComparison(leftSchema, node.getType(), rightSchema)) { throw new KsqlException("Cannot compare " + node.getLeft().toString() + " (" + leftSchema.toString() + ") to " @@ -244,7 +299,7 @@ public Void visitDereferenceExpression( expressionTypeContext.setSqlType(structField.type()); return null; } - + @Override public Void visitStringLiteral( final StringLiteral node, final ExpressionTypeContext expressionTypeContext @@ -472,12 +527,31 @@ public Void visitFunctionCall( final UdfFactory udfFactory = functionRegistry.getUdfFactory(node.getName()); final List argTypes = new ArrayList<>(); + final List newArgTypes = new ArrayList<>(); for (final Expression expression : node.getArguments()) { process(expression, expressionTypeContext); - argTypes.add(expressionTypeContext.getSqlType()); + final SqlType newSqlType = expressionTypeContext.getSqlType(); + argTypes.add(newSqlType); + if (expression instanceof LambdaFunctionCall) { + newArgTypes.add(new SqlArgument(null, SqlLambda.of(expressionTypeContext.getInputTypes(), expressionTypeContext.getSqlType()))); + } else { + newArgTypes.add(new SqlArgument(newSqlType, null)); + } + if (shouldSetInputType(node, expressionTypeContext)) { + if (newSqlType instanceof SqlArray) { + SqlArray inputArray = (SqlArray) newSqlType; + expressionTypeContext.addInputType(inputArray.getItemType()); + } else if (newSqlType instanceof SqlMap) { + SqlMap inputMap = (SqlMap) newSqlType; + expressionTypeContext.addInputType(inputMap.getKeyType()); + expressionTypeContext.addInputType(inputMap.getValueType()); + } else { + expressionTypeContext.addInputType(newSqlType); + } + } } - final SqlType returnSchema = udfFactory.getFunction(argTypes).getReturnType(argTypes); + final SqlType returnSchema = udfFactory.getUdfFunction(newArgTypes).getReturnType(argTypes); expressionTypeContext.setSqlType(returnSchema); return null; } @@ -590,5 +664,11 @@ private Optional validateWhenClauses( return previousResult; } + + private Boolean shouldSetInputType(final FunctionCall node, final ExpressionTypeContext context) { + return (context.getInputTypes() == null) + || (node.getName().equals(FunctionName.of("REDUCE_MAP")) && context.getInputTypes().size() == 2) + || (node.getName().equals(FunctionName.of("REDUCE_ARRAY")) && context.getInputTypes().size() == 1); + } } } diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/CastEvaluatorTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/CastEvaluatorTest.java index 60afa11f4330..f1da6cef9f89 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/CastEvaluatorTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/CastEvaluatorTest.java @@ -81,7 +81,7 @@ */ @RunWith(Enclosed.class) public class CastEvaluatorTest { - + private static final String INNER_CODE = "val0"; @RunWith(Parameterized.class) diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtilTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtilTest.java index d2d504d85eec..826f54a065ab 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtilTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/helpers/LambdaUtilTest.java @@ -20,9 +20,9 @@ import static org.hamcrest.Matchers.is; import com.google.common.collect.ImmutableList; +import io.confluent.ksql.function.TriFunction; import io.confluent.ksql.execution.codegen.CodeGenTestUtil; -import java.util.Arrays; import java.util.List; import java.util.function.BiFunction; import java.util.function.Function; @@ -45,6 +45,7 @@ public void shouldGenerateFunctionCode() { .function(argName, argType, argName + " + 1"); // Then: + System.out.println(javaCode); final Object result = CodeGenTestUtil.cookAndEval(javaCode, Function.class); assertThat(result, is(instanceOf(Function.class))); assertThat(((Function) result).apply(10L), is(11L)); diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/function/UdfUtilTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/function/UdfUtilTest.java index a730a6638d74..4b96d1d0a7d0 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/function/UdfUtilTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/function/UdfUtilTest.java @@ -20,8 +20,11 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.execution.codegen.helpers.TriFunction; import io.confluent.ksql.function.types.ArrayType; import io.confluent.ksql.function.types.GenericType; +import io.confluent.ksql.function.types.LambdaType; import io.confluent.ksql.function.types.MapType; import io.confluent.ksql.function.types.ParamType; import io.confluent.ksql.function.types.ParamTypes; @@ -33,6 +36,8 @@ import java.sql.Timestamp; import java.util.List; import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; import org.apache.kafka.connect.data.Struct; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -233,6 +238,108 @@ public void shouldGetGenericSchemaFromPartialParameterizedType() throws NoSuchMe assertThat(returnType, is(MapType.of(ParamTypes.LONG, GenericType.of("V")))); } + @Test + public void shouldGetFunction() throws NoSuchMethodException { + final Type type = getClass().getDeclaredMethod("functionType", Function.class) + .getGenericParameterTypes()[0]; + final ParamType schema = UdfUtil.getSchemaFromType(type); + assertThat(schema, instanceOf(LambdaType.class)); + assertThat(((LambdaType) schema).inputTypes(), equalTo(ImmutableList.of(ParamTypes.LONG))); + assertThat(((LambdaType) schema).returnType(), equalTo(ParamTypes.INTEGER)); + } + + @Test + public void shouldGetPartialGenericFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("partialGenericFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(ParamTypes.LONG), GenericType.of("U")))); + } + + @Test + public void shouldGetGenericFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("genericFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(GenericType.of("T")), GenericType.of("U")))); + } + + @Test + public void shouldGetBiFunction() throws NoSuchMethodException { + final Type type = getClass().getDeclaredMethod("biFunctionType", BiFunction.class) + .getGenericParameterTypes()[0]; + final ParamType schema = UdfUtil.getSchemaFromType(type); + assertThat(schema, instanceOf(LambdaType.class)); + assertThat(((LambdaType) schema).inputTypes(), equalTo(ImmutableList.of(ParamTypes.LONG, ParamTypes.INTEGER))); + assertThat(((LambdaType) schema).returnType(), equalTo(ParamTypes.BOOLEAN)); + } + + @Test + public void shouldGetPartialGenericBiFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("partialGenericBiFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(GenericType.of("T"), ParamTypes.BOOLEAN), GenericType.of("U")))); + } + + @Test + public void shouldGetGenericBiFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("partialGenericBiFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(GenericType.of("T"), ParamTypes.BOOLEAN), GenericType.of("U")))); + } + + @Test + public void shouldGetTriFunction() throws NoSuchMethodException { + final Type type = getClass().getDeclaredMethod("triFunctionType", TriFunction.class) + .getGenericParameterTypes()[0]; + final ParamType schema = UdfUtil.getSchemaFromType(type); + assertThat(schema, instanceOf(LambdaType.class)); + assertThat(((LambdaType) schema).inputTypes(), equalTo(ImmutableList.of(ParamTypes.LONG, ParamTypes.INTEGER, ParamTypes.BOOLEAN))); + assertThat(((LambdaType) schema).returnType(), equalTo(ParamTypes.BOOLEAN)); + } + + @Test + public void shouldGetPartialGenericTriFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("partialGenericTriFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(GenericType.of("T"), ParamTypes.BOOLEAN, GenericType.of("U")), ParamTypes.INTEGER))); + } + + @Test + public void shouldGetGenericTriFunction() throws NoSuchMethodException { + // Given: + final Type genericType = getClass().getMethod("genericTriFunctionType").getGenericReturnType(); + + // When: + final ParamType returnType = UdfUtil.getSchemaFromType(genericType); + + // Then: + assertThat(returnType, is(LambdaType.of(ImmutableList.of(GenericType.of("T"), GenericType.of("U"), GenericType.of("V")), GenericType.of("W")))); + } + // following methods not invoked but used to test conversion from type -> schema @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) public T genericType() { @@ -261,4 +368,46 @@ private void mapType(final Map map) { @SuppressWarnings("unused") private void listType(final List list) { } + + @SuppressWarnings("unused") + private void functionType(final Function function) { + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public Function partialGenericFunctionType() { + return null; + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public Function genericFunctionType() { + return null; + } + + @SuppressWarnings("unused") + private void biFunctionType(final BiFunction biFunction) { + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public BiFunction partialGenericBiFunctionType() { + return null; + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public BiFunction genericBiFunctionType() { + return null; + } + + @SuppressWarnings("unused") + private void triFunctionType(final TriFunction triFunction) { + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public TriFunction partialGenericTriFunctionType() { + return null; + } + + @SuppressWarnings({"unused", "WeakerAccess", "MethodMayBeStatic"}) + public TriFunction genericTriFunctionType() { + return null; + } } diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/plan.json new file mode 100644 index 000000000000..39320ef6fdd7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/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_ARRAY(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_ARRAY(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" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.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/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/spec.json new file mode 100644 index 000000000000..49fafc2b7df7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/spec.json @@ -0,0 +1,120 @@ +{ + "version" : "6.2.0", + "timestamp" : 1612450676248, + "path" : "query-validation-tests/array_transform.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 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" : [ 10, 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_ARRAY(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/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_apply_lambda_function_to_array/6.2.0_1612450676248/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/array_transform_-_array_max_test/6.2.0_1612454542152/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/plan.json new file mode 100644 index 000000000000..77fcc8336de3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/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_ARRAY(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_ARRAY(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" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.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/array_transform_-_array_max_test/6.2.0_1612454542152/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/spec.json new file mode 100644 index 000000000000..794c91d70c1a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/spec.json @@ -0,0 +1,180 @@ +{ + "version" : "6.2.0", + "timestamp" : 1612454542152, + "path" : "query-validation-tests/array_transform.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 test", + "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" : 100, + "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" : 100, + "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_ARRAY(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/array_transform_-_array_max_test/6.2.0_1612454542152/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_array_max_test/6.2.0_1612454542152/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/array_transform_-_capitalize_string/6.2.0_1612449334271/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/plan.json new file mode 100644 index 000000000000..2dff58992c4d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/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_ARRAY(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_ARRAY(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" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.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/array_transform_-_capitalize_string/6.2.0_1612449334271/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/spec.json new file mode 100644 index 000000000000..afbd1dd8e4b7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/spec.json @@ -0,0 +1,162 @@ +{ + "version" : "6.2.0", + "timestamp" : 1612449334271, + "path" : "query-validation-tests/array_transform.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 string", + "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" : 0, + "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" : 0, + "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_array(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/array_transform_-_capitalize_string/6.2.0_1612449334271/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_capitalize_string/6.2.0_1612449334271/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/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/plan.json new file mode 100644 index 000000000000..f0dd7104ae3d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/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_ARRAY(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_ARRAY(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" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.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/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/spec.json new file mode 100644 index 000000000000..e53a4aa42db5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/spec.json @@ -0,0 +1,174 @@ +{ + "version" : "6.2.0", + "timestamp" : 1612451742541, + "path" : "query-validation-tests/array_transform.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 - 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" : 100, + "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", null ] + } + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "SUM" : [ "TOO LOW", "TOO LOW", "TOO LOW" ] + } + }, { + "topic" : "OUTPUT", + "key" : 100, + "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_ARRAY(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/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_case_check_-_input__double,_output__string/6.2.0_1612451742541/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/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/plan.json new file mode 100644 index 000000000000..30fb604f715b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/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_ARRAY(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_ARRAY(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" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.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/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/spec.json new file mode 100644 index 000000000000..a8780ce3d7db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/spec.json @@ -0,0 +1,201 @@ +{ + "version" : "6.2.0", + "timestamp" : 1612451957459, + "path" : "query-validation-tests/array_transform.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 test", + "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" : 100, + "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" : 100, + "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_ARRAY(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/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/array_transform_-_decimal_absolute_test/6.2.0_1612451957459/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/array_transform.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/array_transform.json new file mode 100644 index 000000000000..95a68939eb75 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/array_transform.json @@ -0,0 +1,98 @@ +{ + "comments": [ + "Tests covering the use of the TRANSFORM_ARRAY function." + ], + "tests": [ + { + "name": "apply 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_ARRAY(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":[10, null]}}, + {"topic": "OUTPUT", "key": "three", "value": {"C":null}} + ] + }, + { + "name": "capitalize 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_array(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": 0, "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": 0,"value": {"LAMBDA": ["KSQLDB", "KAFKA", "STREAMS"]}} + ] + }, + { + "name": "case check - 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_ARRAY(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": 100,"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", null]}}, + {"topic": "OUTPUT", "key": 100,"value": {"SUM": ["TOO LOW", "TOO LOW", "TOO LOW"]}}, + {"topic": "OUTPUT", "key": 100,"value": {"SUM": ["TOO LOW", "TOO LOW", "above 10"]}} + ] + }, + { + "name": "decimal absolute test", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM_ARRAY(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": 100,"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": 100,"value": {"ABS": null}} + ] + }, + { + "name": "array max test", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE ARRAY>) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE STREAM OUTPUT as SELECT ID, TRANSFORM_ARRAY(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": 100,"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": 100,"value": {"MAX": null}} + ] + } + ] +} \ No newline at end of file diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java index 9f1fcf948c81..a0a502d205b3 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java @@ -181,16 +181,23 @@ public EndpointResponse streamQuery( final Optional isInternalRequest, final KsqlMediaType mediaType ) { - throwIfNotConfigured(); - activenessRegistrar.updateLastRequestTime(); + EndpointResponse response = null; + try { + throwIfNotConfigured(); + activenessRegistrar.updateLastRequestTime(); - final PreparedStatement statement = parseStatement(request); + final PreparedStatement statement = parseStatement(request); - CommandStoreUtil.httpWaitForCommandSequenceNumber( - commandQueue, request, commandQueueCatchupTimeout); + CommandStoreUtil.httpWaitForCommandSequenceNumber( + commandQueue, request, commandQueueCatchupTimeout); - return handleStatement(securityContext, request, statement, connectionClosedFuture, - isInternalRequest, mediaType); + response = handleStatement(securityContext, request, statement, connectionClosedFuture, + isInternalRequest, mediaType); + } catch (final Exception e) { + e.printStackTrace(); + throw e; + } + return response; } private void throwIfNotConfigured() { diff --git a/ksqldb-udf/src/main/java/io/confluent/ksql/function/TriFunction.java b/ksqldb-udf/src/main/java/io/confluent/ksql/function/TriFunction.java new file mode 100644 index 000000000000..03d21f3374ee --- /dev/null +++ b/ksqldb-udf/src/main/java/io/confluent/ksql/function/TriFunction.java @@ -0,0 +1,31 @@ +/* + * 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; + +import java.util.Objects; +import java.util.function.Function; + +@FunctionalInterface +public interface TriFunction { + + R apply(T a, U b, V c); + + default TriFunction andThen( + Function after) { + Objects.requireNonNull(after); + return (T t, U u, V v) -> after.apply(apply(t, u, v)); + } +} diff --git a/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/SqlArgument.java b/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/SqlArgument.java new file mode 100644 index 000000000000..47a0234306f3 --- /dev/null +++ b/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/SqlArgument.java @@ -0,0 +1,39 @@ +/* + * 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.schema.ksql; + +import io.confluent.ksql.schema.ksql.types.SqlLambda; +import io.confluent.ksql.schema.ksql.types.SqlType; + +public class SqlArgument { + + private final SqlType sqlType; + private final SqlLambda sqlLambda; + + public SqlArgument(final SqlType type, final SqlLambda lambda) { + sqlType = type; + sqlLambda = lambda; + } + + public SqlType getSqlType() { + return sqlType; + } + + public SqlLambda getSqlLambda() { + return sqlLambda; + } + +} diff --git a/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/types/SqlLambda.java b/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/types/SqlLambda.java new file mode 100644 index 000000000000..a8bbbf6c0356 --- /dev/null +++ b/ksqldb-udf/src/main/java/io/confluent/ksql/schema/ksql/types/SqlLambda.java @@ -0,0 +1,77 @@ +/* + * 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.schema.ksql.types; + +import static java.util.Objects.requireNonNull; + +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.schema.utils.FormatOptions; +import java.util.List; +import java.util.Objects; + +@Immutable +public final class SqlLambda { + + private final List inputTypes; + private final SqlType returnType; + + public static SqlLambda of( + final List inputType, + final SqlType returnType + ) { + return new SqlLambda(inputType, returnType); + } + + public SqlLambda( + final List inputTypes, + final SqlType returnType + ) { + this.inputTypes = requireNonNull(inputTypes, "inputType"); + this.returnType = requireNonNull(returnType, "returnType"); + } + + public List getInputType() { + return inputTypes; + } + + public SqlType getReturnType() { + return returnType; + } + + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final SqlLambda lambda = (SqlLambda) o; + return Objects.equals(inputTypes, lambda.inputTypes) + && Objects.equals(returnType, lambda.returnType); + } + + public int hashCode() { + return Objects.hash(inputTypes, returnType); + } + + public String toString() { + return toString(FormatOptions.none()); + } + + public String toString(final FormatOptions formatOptions) { + return "Lambda<" + inputTypes + ", " + returnType + ">"; + } +}