You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/10/06 08:09:05 UTC

[GitHub] [flink] dianfu commented on a diff in pull request #20947: [FLINK-29353]Support UNIX_TIMESTAMP built-in function in Table API

dianfu commented on code in PR #20947:
URL: https://github.com/apache/flink/pull/20947#discussion_r988682409


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java:
##########
@@ -1548,6 +1548,20 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
                     .outputTypeStrategy(nullableIfArgs(explicit(STRING())))
                     .build();
 
+    public static final BuiltInFunctionDefinition UNIX_TIMESTAMP =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("unixTimestamp")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            or(
+                                    NO_ARGS,
+                                    sequence(logical(LogicalTypeFamily.CHARACTER_STRING)),
+                                    sequence(
+                                            logical(LogicalTypeFamily.CHARACTER_STRING),
+                                            logical(LogicalTypeFamily.CHARACTER_STRING))))
+                    .outputTypeStrategy(nullableIfArgs(explicit(BIGINT())))

Review Comment:
   ```suggestion
                       .outputTypeStrategy(explicit(BIGINT().nullable()))
   ```



##########
flink-python/pyflink/table/tests/test_expression.py:
##########
@@ -275,6 +275,10 @@ def test_expressions(self):
         self.assertEqual("fromUnixtime(1)", str(from_unixtime(1)))
         self.assertEqual("fromUnixtime(1, 'yy-MM-dd HH-mm-ss')",
                          str(from_unixtime(1, 'yy-MM-dd HH-mm-ss')))
+        self.assertEqual("unixTimestamp('2015-07-24 10:00:00')",

Review Comment:
   Also test the cases when there is no argument?



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java:
##########
@@ -412,6 +412,39 @@ public static ApiExpression fromUnixtime(Object unixtime, Object format) {
         return apiCall(BuiltInFunctionDefinitions.FROM_UNIXTIME, unixtime, format);
     }
 
+    /**
+     * Gets current Unix timestamp in seconds. This function is not deterministic which means the
+     * value would be recalculated for each record.
+     *
+     * @return The formatted timestamp as string.
+     */
+    public static ApiExpression unixTimestamp() {
+        return apiCall(BuiltInFunctionDefinitions.UNIX_TIMESTAMP);
+    }
+
+    /**
+     * Converts date time string string1 in format string2 (by default: yyyy-MM-dd HH:mm:ss if not

Review Comment:
   It has only one argument and so need to adjust the documentation accordingly.



##########
flink-python/pyflink/table/expressions.py:
##########
@@ -358,6 +358,19 @@ def from_unixtime(unixtime, format=None) -> Expression:
         return _binary_op("fromUnixtime", unixtime, format)
 
 
+def unix_timestamp(date_str: str = None, format: str = None) -> Expression:
+    """
+    Converts date time string string1 in format string2 (by default: yyyy-MM-dd HH:mm:ss if not

Review Comment:
   Also explain the cases when all arguments are None?



##########
docs/data/sql_functions.yml:
##########
@@ -491,8 +491,10 @@ temporal:
     table: fromUnixtime(NUMERIC[, STRING])
     description: Returns a representation of the numeric argument as a value in string format (default is 'yyyy-MM-dd HH:mm:ss'). numeric is an internal timestamp value representing seconds since '1970-01-01 00:00:00' UTC, such as produced by the UNIX_TIMESTAMP() function. The return value is expressed in the session time zone (specified in TableConfig). E.g., FROM_UNIXTIME(44) returns '1970-01-01 00:00:44' if in UTC time zone, but returns '1970-01-01 09:00:44' if in 'Asia/Tokyo' time zone.
   - sql: UNIX_TIMESTAMP()
+    table: unixtimestamp()

Review Comment:
   ```suggestion
       table: unixTimestamp()
   ```



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala:
##########
@@ -931,6 +931,17 @@ class TemporalTypesTest extends ExpressionTestBase {
     val ss2 = "2015-07-25 02:02:02"
     val fmt = "yyyy/MM/dd HH:mm:ss.S"
 
+    testAllApis(unixTimestamp(ss1), s"UNIX_TIMESTAMP('$ss1')", (ts1.getTime / 1000L).toString)
+    testAllApis(unixTimestamp(ss2), s"UNIX_TIMESTAMP('$ss2')", (ts2.getTime / 1000L).toString)
+    testAllApis(
+      unixTimestamp(s1, fmt),
+      s"UNIX_TIMESTAMP('$s1', '$fmt')",
+      (ts1.getTime / 1000L).toString)
+    testAllApis(
+      unixTimestamp(s2, fmt),
+      s"UNIX_TIMESTAMP('$s2', '$fmt')",
+      (ts2.getTime / 1000L).toString)
+
     testSqlApi(s"UNIX_TIMESTAMP('$ss1')", (ts1.getTime / 1000L).toString)

Review Comment:
   Remove the following testSqlApis as they will be covered in the added tests



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org