twalthr commented on code in PR #23811: URL: https://github.com/apache/flink/pull/23811#discussion_r1413957004
########## flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ExpressionSerializationTest.java: ########## @@ -0,0 +1,365 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.expressions; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.JsonExistsOnError; +import org.apache.flink.table.api.JsonOnNull; +import org.apache.flink.table.api.JsonQueryOnEmptyOrError; +import org.apache.flink.table.api.JsonQueryWrapper; +import org.apache.flink.table.api.JsonType; +import org.apache.flink.table.api.JsonValueOnEmptyOrError; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.expressions.resolver.ExpressionResolver; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.operations.ValuesQueryOperation; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; +import org.apache.flink.table.utils.FunctionLookupMock; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Stream; + +import static org.apache.flink.table.api.Expressions.$; +import static org.apache.flink.table.api.Expressions.negative; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for serializing {@link BuiltInFunctionDefinitions} into a SQL string. */ +public class ExpressionSerializationTest { + + public static Stream<TestSpec> testData() { + return Stream.of( + TestSpec.forExpr(Expressions.uuid()).expectStr("UUID()"), + TestSpec.forExpr($("f0").abs()) + .withField("f0", DataTypes.BIGINT()) + .expectStr("ABS(`f0`)"), + TestSpec.forExpr($("f0").isLess(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` < 123"), + TestSpec.forExpr($("f0").isLessOrEqual(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` <= 123"), + TestSpec.forExpr($("f0").isEqual(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` = 123"), + TestSpec.forExpr($("f0").isNotEqual(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` <> 123"), + TestSpec.forExpr($("f0").isGreaterOrEqual(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` >= 123"), + TestSpec.forExpr($("f0").isGreater(123)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` > 123"), + TestSpec.forExpr($("f0").isNull()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS NULL"), + TestSpec.forExpr($("f0").isNotNull()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS NOT NULL"), + TestSpec.forExpr($("f0").isTrue()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS TRUE"), + TestSpec.forExpr($("f0").isNotTrue()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS NOT TRUE"), + TestSpec.forExpr($("f0").isFalse()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS FALSE"), + TestSpec.forExpr($("f0").isNotFalse()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("`f0` IS NOT FALSE"), + TestSpec.forExpr($("f0").not()) + .withField("f0", DataTypes.BOOLEAN()) + .expectStr("NOT `f0`"), + TestSpec.forExpr( + Expressions.and( + $("f0").isNotNull(), + $("f0").isLess(420), + $("f0").isGreater(123))) + .withField("f0", DataTypes.BIGINT()) + .expectStr("(`f0` IS NOT NULL) AND (`f0` < 420) AND (`f0` > 123)"), + TestSpec.forExpr( + Expressions.or( + $("f0").isNotNull(), + $("f0").isLess(420), + $("f0").isGreater(123))) + .withField("f0", DataTypes.BIGINT()) + .expectStr("(`f0` IS NOT NULL) OR (`f0` < 420) OR (`f0` > 123)"), + TestSpec.forExpr( + Expressions.ifThenElse( + $("f0").isNotNull(), $("f0").plus(420), $("f0").minus(123))) + .withField("f0", DataTypes.BIGINT()) + .expectStr( + "CASE WHEN `f0` IS NOT NULL THEN `f0` + 420 ELSE `f0` - 123 END"), + TestSpec.forExpr($("f0").times(3).dividedBy($("f1"))) + .withField("f0", DataTypes.BIGINT()) + .withField("f1", DataTypes.BIGINT()) + .expectStr("(`f0` * 3) / `f1`"), + TestSpec.forExpr($("f0").mod(5)) + .withField("f0", DataTypes.BIGINT()) + .expectStr("`f0` % 5"), + TestSpec.forExpr(negative($("f0"))) Review Comment: very nit: all other ones are prefixed with `Expressions.` ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/TypeLiteralExpression.java: ########## @@ -57,6 +58,12 @@ public String asSummaryString() { return dataType.toString(); } + @Override + public String asSerializableString() { + final LogicalType logicalType = dataType.getLogicalType(); + return logicalType.copy(true).asSerializableString(); Review Comment: maybe add a comment why we are removing the nullability. this is a bit inconsistent because we clearly loose information during serialization. but I agree with this approach. ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java: ########## @@ -216,6 +219,35 @@ public String asSummaryString() { return getFunctionName() + argList; } + @Override + public String asSerializableString() { + if (functionDefinition instanceof BuiltInFunctionDefinition) { + final BuiltInFunctionDefinition definition = + (BuiltInFunctionDefinition) functionDefinition; + return definition.getCallSyntax().unparse(definition.getSqlName(), args); + } else { + return SqlCallSyntax.FUNCTION.unparse(getSerializableFunctionName(), args); + } + } + + private String getSerializableFunctionName() { + if (functionIdentifier == null) { + throw new TableException( + "Only functions that have been registered before are serializable."); + } + + return functionIdentifier + .getIdentifier() + .map( + i -> + String.format( Review Comment: ObjectIdentifier has already a method that can do the serialization. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java: ########## @@ -145,6 +152,27 @@ private static List<Row> materializeResult(TableResult tableResult) { // --------------------------------------------------------------------------------------------- + protected static final class TableApiQuerySpec { Review Comment: nit: maybe `TableApiAggSpec` to give the class name more meaning? ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java: ########## @@ -216,6 +219,35 @@ public String asSummaryString() { return getFunctionName() + argList; } + @Override + public String asSerializableString() { + if (functionDefinition instanceof BuiltInFunctionDefinition) { + final BuiltInFunctionDefinition definition = + (BuiltInFunctionDefinition) functionDefinition; + return definition.getCallSyntax().unparse(definition.getSqlName(), args); + } else { + return SqlCallSyntax.FUNCTION.unparse(getSerializableFunctionName(), args); + } + } + + private String getSerializableFunctionName() { + if (functionIdentifier == null) { + throw new TableException( + "Only functions that have been registered before are serializable."); + } + + return functionIdentifier + .getIdentifier() + .map( + i -> + String.format( + "%s.%s.%s", + EncodingUtils.escapeIdentifier(i.getCatalogName()), + EncodingUtils.escapeIdentifier(i.getDatabaseName()), + EncodingUtils.escapeIdentifier(i.getObjectName()))) + .orElseGet(functionIdentifier::getFunctionName); Review Comment: also run `escapeIdentifier` on function names -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org