This is an automated email from the ASF dual-hosted git repository.

fhueske pushed a commit to branch 
fhueske-FLINK-36642-Fix_SqlNames_of_builtin_functions
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e49564a2e36a43ac5a3cb4bd64c0470f3ec9c9df
Author: Fabian Hueske <[email protected]>
AuthorDate: Wed Oct 30 19:46:32 2024 +0100

    [FLINK-36642][table] Fix SQL names of built-in functions
---
 .../expressions/ExpressionSerializationTest.java   | 71 +++++++++++++++++++++-
 .../functions/BuiltInFunctionDefinitions.java      | 23 +++++++
 2 files changed, 93 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 d9e9728f9c4..382f9f0e3db 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.assertj.core.api.Assertions.assertThat;
 
 /** Tests for serializing {@link BuiltInFunctionDefinitions} into a SQL 
string. */
@@ -128,6 +129,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)"),
@@ -193,6 +196,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").extract(TimeIntervalUnit.HOUR))
                         .withField("f0", DataTypes.TIMESTAMP())
                         .expectStr("EXTRACT(HOUR FROM `f0`)"),
@@ -211,6 +235,38 @@ public class ExpressionSerializationTest {
                         .withField("f2", DataTypes.TIMESTAMP())
                         .withField("f3", DataTypes.TIMESTAMP())
                         .expectStr("(`f0`, `f1`) OVERLAPS (`f2`, `f3`)"),
+                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()"),
                 
TestSpec.forExpr($("f0").get("g0").plus($("f0").get("g1").get("h1")))
                         .withField(
                                 "f0",
@@ -306,7 +362,20 @@ public class ExpressionSerializationTest {
                                         .plus($("f0").max()))
                         .withField("f0", DataTypes.BIGINT())
                         .expectStr(
-                                "((COUNT(DISTINCT `f0`)) + (AVG(DISTINCT 
`f0`))) + (MAX(`f0`))"));
+                                "((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`)")
+            );
     }
 
     @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 a6ab7984457..9e6701b3529 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
@@ -809,6 +809,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(
@@ -818,6 +819,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(
@@ -827,6 +829,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(
@@ -836,6 +839,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(
@@ -894,6 +898,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())))
@@ -1286,6 +1291,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())))
@@ -1294,6 +1300,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())))
@@ -1389,6 +1396,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition PARSE_URL =
             BuiltInFunctionDefinition.newBuilder()
                     .name("parseUrl")
+                    .sqlName("PARSE_URL")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -1494,6 +1502,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition REGEXP_REPLACE =
             BuiltInFunctionDefinition.newBuilder()
                     .name("regexpReplace")
+                    .sqlName("REGEXP_REPLACE")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             sequence(
@@ -1514,6 +1523,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition SPLIT_INDEX =
             BuiltInFunctionDefinition.newBuilder()
                     .name("splitIndex")
+                    .sqlName("SPLIT_INDEX")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             sequence(
@@ -1526,6 +1536,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition STR_TO_MAP =
             BuiltInFunctionDefinition.newBuilder()
                     .name("strToMap")
+                    .sqlName("STR_TO_MAP")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -2028,6 +2039,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition RAND_INTEGER =
             BuiltInFunctionDefinition.newBuilder()
                     .name("randInteger")
+                    .sqlName("RAND_INTEGER")
                     .kind(SCALAR)
                     .notDeterministic()
                     .inputTypeStrategy(
@@ -2122,6 +2134,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();
@@ -2129,6 +2142,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();
@@ -2143,6 +2157,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();
@@ -2157,6 +2172,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition CURRENT_ROW_TIMESTAMP =
             BuiltInFunctionDefinition.newBuilder()
                     .name("currentRowTimestamp")
+                    .name("CURRENT_ROW_TIMESTAMP")
                     .kind(SCALAR)
                     .outputTypeStrategy(explicit(TIMESTAMP_LTZ(3).notNull()))
                     .notDeterministic()
@@ -2188,6 +2204,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition DATE_FORMAT =
             BuiltInFunctionDefinition.newBuilder()
                     .name("dateFormat")
+                    .sqlName("DATE_FORMAT")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -2223,6 +2240,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition CONVERT_TZ =
             BuiltInFunctionDefinition.newBuilder()
                     .name("convertTz")
+                    .sqlName("CONVERT_TZ")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             sequence(
@@ -2235,6 +2253,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition FROM_UNIXTIME =
             BuiltInFunctionDefinition.newBuilder()
                     .name("fromUnixtime")
+                    .sqlName("FROM_UNIXTIME")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -2248,6 +2267,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition UNIX_TIMESTAMP =
             BuiltInFunctionDefinition.newBuilder()
                     .name("unixTimestamp")
+                    .sqlName("UNIX_TIMESTAMP")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -2262,6 +2282,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition TO_DATE =
             BuiltInFunctionDefinition.newBuilder()
                     .name("toDate")
+                    .sqlName("TO_DATE")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(
@@ -2275,6 +2296,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition TO_TIMESTAMP_LTZ =
             BuiltInFunctionDefinition.newBuilder()
                     .name("toTimestampLtz")
+                    .sqlName("TO_TIMESTAMP_LTZ")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             sequence(
@@ -2286,6 +2308,7 @@ public final class BuiltInFunctionDefinitions {
     public static final BuiltInFunctionDefinition TO_TIMESTAMP =
             BuiltInFunctionDefinition.newBuilder()
                     .name("toTimestamp")
+                    .sqlName("TO_TIMESTAMP")
                     .kind(SCALAR)
                     .inputTypeStrategy(
                             or(

Reply via email to