This is an automated email from the ASF dual-hosted git repository. jark pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit b41b52a6d1a18eafa113de3abd14da519de8cf2f Author: Jark Wu <imj...@gmail.com> AuthorDate: Wed Aug 7 13:41:31 2019 +0800 [FLINK-13561][table-planner-blink] Drop TO_TIMESTAMP(bigint) function support This commit drops TO_TIMESTAMP(bigint) function support in blink planner to align with other systems. We only support TO_TIMESTAMP(string [,format]) in this version. --- .../functions/sql/FlinkSqlOperatorTable.java | 5 ++-- .../planner/expressions/ScalarFunctionsTest.scala | 34 ++-------------------- .../table/runtime/functions/SqlDateTimeUtils.java | 4 ++- 3 files changed, 8 insertions(+), 35 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index e27e458..4c702a4 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -761,9 +761,8 @@ public class FlinkSqlOperatorTable extends ReflectiveSqlOperatorTable { ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.TIMESTAMP), SqlTypeTransforms.FORCE_NULLABLE), null, OperandTypes.or( - OperandTypes.family(SqlTypeFamily.NUMERIC), - OperandTypes.family(SqlTypeFamily.STRING), - OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING)), + OperandTypes.family(SqlTypeFamily.CHARACTER), + OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER)), SqlFunctionCategory.TIMEDATE); public static final SqlFunction TO_DATE = new SqlFunction( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala index 244812a..0e5f120 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala @@ -3531,6 +3531,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { testSqlApi("to_timestamp(1513135677000)", "2017-12-13 03:27:57.000") testSqlApi("to_timestamp('2017-09-15 00:00:00')", "2017-09-15 00:00:00.000") testSqlApi("to_timestamp('20170915000000', 'yyyyMMddHHmmss')", "2017-09-15 00:00:00.000") + testSqlApi("to_timestamp('2017-09-15', 'yyyy-MM-dd')", "2017-09-15 00:00:00.000") + // test with null input + testSqlApi("to_timestamp(cast(null as varchar))", "null") } @Test @@ -3989,37 +3992,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { } @Test - def testToTimestampWithNumeric(): Unit = { - // Test integral and fractional numeric to timestamp. - testSqlApi( - "to_timestamp(f2)", - "1970-01-01 00:00:00.042") - testSqlApi( - "to_timestamp(f3)", - "1970-01-01 00:00:00.043") - testSqlApi( - "to_timestamp(f4)", - "1970-01-01 00:00:00.044") - testSqlApi( - "to_timestamp(f5)", - "1970-01-01 00:00:00.004") - testSqlApi( - "to_timestamp(f6)", - "1970-01-01 00:00:00.004") - testSqlApi( - "to_timestamp(f7)", - "1970-01-01 00:00:00.003") - // Test decimal to timestamp. - testSqlApi( - "to_timestamp(f15)", - "1969-12-31 23:59:58.769") - // test with null input - testSqlApi( - "to_timestamp(cast(null as varchar))", - "null") - } - - @Test def testFromUnixTimeWithNumeric(): Unit = { // Test integral and fractional numeric from_unixtime. testSqlApi( diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index 3cd1246..a6784bb 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -220,7 +220,9 @@ public class SqlDateTimeUtils { public static Long toTimestamp(String dateStr, TimeZone tz) { int length = dateStr.length(); String format; - if (length == 21) { + if (length == 10) { + format = DATE_FORMAT_STRING; + } else if (length == 21) { format = DEFAULT_DATETIME_FORMATS[1]; } else if (length == 22) { format = DEFAULT_DATETIME_FORMATS[2];