This is an automated email from the ASF dual-hosted git repository. fhueske pushed a commit to branch fhueske-FLINK-36642-backport in repository https://gitbox.apache.org/repos/asf/flink.git
commit 1c51e884ff5824c2bb41bff3bdbc038a296f6022 Author: Fabian Hueske <fhue...@confluent.io> AuthorDate: Mon Nov 4 12:21:45 2024 +0100 [FLINK-36642][table] Fix SQL names of built-in functions (#25601) * fix sqlNames of built-in functions * add test cases to ExpressionSerializationTest checking correct SQL serialization of Table API expression --- .../expressions/ExpressionSerializationTest.java | 66 +++++++++++++++++++++- .../functions/BuiltInFunctionDefinitions.java | 23 ++++++++ 2 files changed, 88 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ExpressionSerializationTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ExpressionSerializationTest.java index 22d85bad8eb..cee99c63273 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ExpressionSerializationTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ExpressionSerializationTest.java @@ -48,6 +48,7 @@ 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.lit; import static org.apache.flink.table.api.Expressions.timestampDiff; import static org.assertj.core.api.Assertions.assertThat; @@ -129,6 +130,8 @@ public class ExpressionSerializationTest { TestSpec.forExpr(Expressions.negative($("f0"))) .withField("f0", DataTypes.BIGINT()) .expectStr("- `f0`"), + TestSpec.forExpr(Expressions.randInteger(Expressions.lit(10))) + .expectStr("RAND_INTEGER(10)"), TestSpec.forExpr($("f0").in(1, 2, 3, 4, 5)) .withField("f0", DataTypes.INT()) .expectStr("`f0` IN (1, 2, 3, 4, 5)"), @@ -194,6 +197,27 @@ public class ExpressionSerializationTest { TestSpec.forExpr($("f0").substring(2, 5)) .withField("f0", DataTypes.STRING()) .expectStr("SUBSTRING(`f0` FROM 2 FOR 5)"), + TestSpec.forExpr($("f0").charLength()) + .withField("f0", DataTypes.STRING()) + .expectStr("CHAR_LENGTH(`f0`)"), + TestSpec.forExpr($("f0").fromBase64()) + .withField("f0", DataTypes.STRING()) + .expectStr("FROM_BASE64(`f0`)"), + TestSpec.forExpr($("f0").toBase64()) + .withField("f0", DataTypes.STRING()) + .expectStr("TO_BASE64(`f0`)"), + TestSpec.forExpr($("f0").parseUrl(lit("HOST"))) + .withField("f0", DataTypes.STRING()) + .expectStr("PARSE_URL(`f0`, 'HOST')"), + TestSpec.forExpr($("f0").regexpReplace(lit("[0-9]"), lit("$"))) + .withField("f0", DataTypes.STRING()) + .expectStr("REGEXP_REPLACE(`f0`, '[0-9]', '$')"), + TestSpec.forExpr($("f0").splitIndex(lit(":"), lit(2))) + .withField("f0", DataTypes.STRING()) + .expectStr("SPLIT_INDEX(`f0`, ':', 2)"), + TestSpec.forExpr($("f0").strToMap()) + .withField("f0", DataTypes.STRING()) + .expectStr("STR_TO_MAP(`f0`)"), TestSpec.forExpr($("f0").get("g0").plus($("f0").get("g1").get("h1"))) .withField( "f0", @@ -289,6 +313,18 @@ public class ExpressionSerializationTest { .plus($("f0").max())) .withField("f0", DataTypes.BIGINT()) .expectStr("((COUNT(DISTINCT `f0`)) + (AVG(DISTINCT `f0`))) + (MAX(`f0`))"), + TestSpec.forExpr($("f0").stddevPop()) + .withField("f0", DataTypes.BIGINT()) + .expectStr("STDDEV_POP(`f0`)"), + TestSpec.forExpr($("f0").stddevSamp()) + .withField("f0", DataTypes.BIGINT()) + .expectStr("STDDEV_SAMP(`f0`)"), + TestSpec.forExpr($("f0").varPop()) + .withField("f0", DataTypes.BIGINT()) + .expectStr("VAR_POP(`f0`)"), + TestSpec.forExpr($("f0").varSamp()) + .withField("f0", DataTypes.BIGINT()) + .expectStr("VAR_SAMP(`f0`)"), // Time functions TestSpec.forExpr($("f0").extract(TimeIntervalUnit.HOUR)) @@ -312,7 +348,35 @@ public class ExpressionSerializationTest { TestSpec.forExpr(timestampDiff(TimePointUnit.DAY, $("f0"), $("f1"))) .withField("f0", DataTypes.TIMESTAMP()) .withField("f1", DataTypes.TIMESTAMP()) - .expectStr("TIMESTAMPDIFF(DAY, `f0`, `f1`)")); + .expectStr("TIMESTAMPDIFF(DAY, `f0`, `f1`)"), + TestSpec.forExpr(Expressions.currentDate()).expectStr("CURRENT_DATE()"), + TestSpec.forExpr(Expressions.currentTime()).expectStr("CURRENT_TIME()"), + TestSpec.forExpr(Expressions.currentTimestamp()).expectStr("CURRENT_TIMESTAMP()"), + TestSpec.forExpr(Expressions.dateFormat($("f0"), lit("yyyy-MM-dd"))) + .withField("f0", DataTypes.TIMESTAMP(3)) + .expectStr("DATE_FORMAT(`f0`, 'yyyy-MM-dd')"), + TestSpec.forExpr(Expressions.toTimestamp($("f0"))) + .withField("f0", DataTypes.STRING()) + .expectStr("TO_TIMESTAMP(`f0`)"), + TestSpec.forExpr(Expressions.toTimestampLtz($("f0"), lit(3))) + .withField("f0", DataTypes.BIGINT()) + .expectStr("TO_TIMESTAMP_LTZ(`f0`, 3)"), + TestSpec.forExpr($("f0").toDate()) + .withField("f0", DataTypes.STRING()) + .expectStr("CAST(`f0` AS DATE)"), + TestSpec.forExpr($("f0").toTime()) + .withField("f0", DataTypes.STRING()) + .expectStr("CAST(`f0` AS TIME(0))"), + TestSpec.forExpr($("f0").toTimestamp()) + .withField("f0", DataTypes.STRING()) + .expectStr("CAST(`f0` AS TIMESTAMP(3))"), + TestSpec.forExpr(Expressions.convertTz($("f0"), lit("PST"), lit("GMT"))) + .withField("f0", DataTypes.STRING()) + .expectStr("CONVERT_TZ(`f0`, 'PST', 'GMT')"), + TestSpec.forExpr(Expressions.fromUnixtime($("f0"))) + .withField("f0", DataTypes.BIGINT()) + .expectStr("FROM_UNIXTIME(`f0`)"), + TestSpec.forExpr(Expressions.unixTimestamp()).expectStr("UNIX_TIMESTAMP()")); } @ParameterizedTest diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index 7aed83fcf6f..80204ceeff6 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -787,6 +787,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition STDDEV_POP = BuiltInFunctionDefinition.newBuilder() .name("stddevPop") + .sqlName("STDDEV_POP") .kind(AGGREGATE) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.NUMERIC))) .outputTypeStrategy( @@ -796,6 +797,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition STDDEV_SAMP = BuiltInFunctionDefinition.newBuilder() .name("stddevSamp") + .sqlName("STDDEV_SAMP") .kind(AGGREGATE) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.NUMERIC))) .outputTypeStrategy( @@ -805,6 +807,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition VAR_POP = BuiltInFunctionDefinition.newBuilder() .name("varPop") + .sqlName("VAR_POP") .kind(AGGREGATE) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.NUMERIC))) .outputTypeStrategy( @@ -814,6 +817,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition VAR_SAMP = BuiltInFunctionDefinition.newBuilder() .name("varSamp") + .sqlName("VAR_SAMP") .kind(AGGREGATE) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.NUMERIC))) .outputTypeStrategy( @@ -851,6 +855,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CHAR_LENGTH = BuiltInFunctionDefinition.newBuilder() .name("charLength") + .sqlName("CHAR_LENGTH") .kind(SCALAR) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.CHARACTER_STRING))) .outputTypeStrategy(nullableIfArgs(explicit(INT()))) @@ -1090,6 +1095,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition FROM_BASE64 = BuiltInFunctionDefinition.newBuilder() .name("fromBase64") + .sqlName("FROM_BASE64") .kind(SCALAR) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.CHARACTER_STRING))) .outputTypeStrategy(nullableIfArgs(explicit(DataTypes.STRING()))) @@ -1098,6 +1104,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition TO_BASE64 = BuiltInFunctionDefinition.newBuilder() .name("toBase64") + .sqlName("TO_BASE64") .kind(SCALAR) .inputTypeStrategy(sequence(logical(LogicalTypeFamily.CHARACTER_STRING))) .outputTypeStrategy(nullableIfArgs(explicit(DataTypes.STRING()))) @@ -1193,6 +1200,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition PARSE_URL = BuiltInFunctionDefinition.newBuilder() .name("parseUrl") + .sqlName("PARSE_URL") .kind(SCALAR) .inputTypeStrategy( or( @@ -1266,6 +1274,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition REGEXP_REPLACE = BuiltInFunctionDefinition.newBuilder() .name("regexpReplace") + .sqlName("REGEXP_REPLACE") .kind(SCALAR) .inputTypeStrategy( sequence( @@ -1286,6 +1295,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition SPLIT_INDEX = BuiltInFunctionDefinition.newBuilder() .name("splitIndex") + .sqlName("SPLIT_INDEX") .kind(SCALAR) .inputTypeStrategy( sequence( @@ -1298,6 +1308,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition STR_TO_MAP = BuiltInFunctionDefinition.newBuilder() .name("strToMap") + .sqlName("STR_TO_MAP") .kind(SCALAR) .inputTypeStrategy( or( @@ -1769,6 +1780,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition RAND_INTEGER = BuiltInFunctionDefinition.newBuilder() .name("randInteger") + .sqlName("RAND_INTEGER") .kind(SCALAR) .notDeterministic() .inputTypeStrategy( @@ -1850,6 +1862,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CURRENT_DATE = BuiltInFunctionDefinition.newBuilder() .name("currentDate") + .sqlName("CURRENT_DATE") .kind(SCALAR) .outputTypeStrategy(explicit(DATE().notNull())) .build(); @@ -1857,6 +1870,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CURRENT_TIME = BuiltInFunctionDefinition.newBuilder() .name("currentTime") + .sqlName("CURRENT_TIME") .kind(SCALAR) .outputTypeStrategy(explicit(TIME().notNull())) .build(); @@ -1871,6 +1885,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CURRENT_TIMESTAMP = BuiltInFunctionDefinition.newBuilder() .name("currentTimestamp") + .sqlName("CURRENT_TIMESTAMP") .kind(SCALAR) .outputTypeStrategy(explicit(TIMESTAMP_LTZ(3).notNull())) .build(); @@ -1885,6 +1900,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CURRENT_ROW_TIMESTAMP = BuiltInFunctionDefinition.newBuilder() .name("currentRowTimestamp") + .sqlName("CURRENT_ROW_TIMESTAMP") .kind(SCALAR) .outputTypeStrategy(explicit(TIMESTAMP_LTZ(3).notNull())) .notDeterministic() @@ -1916,6 +1932,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition DATE_FORMAT = BuiltInFunctionDefinition.newBuilder() .name("dateFormat") + .sqlName("DATE_FORMAT") .kind(SCALAR) .inputTypeStrategy( or( @@ -1963,6 +1980,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition CONVERT_TZ = BuiltInFunctionDefinition.newBuilder() .name("convertTz") + .sqlName("CONVERT_TZ") .kind(SCALAR) .inputTypeStrategy( sequence( @@ -1975,6 +1993,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition FROM_UNIXTIME = BuiltInFunctionDefinition.newBuilder() .name("fromUnixtime") + .sqlName("FROM_UNIXTIME") .kind(SCALAR) .inputTypeStrategy( or( @@ -1988,6 +2007,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition UNIX_TIMESTAMP = BuiltInFunctionDefinition.newBuilder() .name("unixTimestamp") + .sqlName("UNIX_TIMESTAMP") .kind(SCALAR) .inputTypeStrategy( or( @@ -2002,6 +2022,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition TO_DATE = BuiltInFunctionDefinition.newBuilder() .name("toDate") + .sqlName("TO_DATE") .kind(SCALAR) .inputTypeStrategy( or( @@ -2015,6 +2036,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition TO_TIMESTAMP_LTZ = BuiltInFunctionDefinition.newBuilder() .name("toTimestampLtz") + .sqlName("TO_TIMESTAMP_LTZ") .kind(SCALAR) .inputTypeStrategy( sequence( @@ -2026,6 +2048,7 @@ public final class BuiltInFunctionDefinitions { public static final BuiltInFunctionDefinition TO_TIMESTAMP = BuiltInFunctionDefinition.newBuilder() .name("toTimestamp") + .sqlName("TO_TIMESTAMP") .kind(SCALAR) .inputTypeStrategy( or(