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/04/08 14:26:37 UTC

[GitHub] [flink] twalthr commented on a diff in pull request #18179: [FLINK-8518][Table SQL / API] Add support for extract Epoch

twalthr commented on code in PR #18179:
URL: https://github.com/apache/flink/pull/18179#discussion_r846157113


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/SymbolUtil.java:
##########
@@ -184,6 +184,27 @@
         addSymbolMapping(null, null, SqlJsonEmptyOrError.ERROR, "JSON_EMPTY_OR_ERROR", "ERROR");
 
         // TIME_UNIT_RANGE
+        addSymbolMapping(
+                TimeIntervalUnit.MILLENNIUM,
+                DateTimeUtils.TimeUnitRange.MILLENNIUM,
+                TimeUnitRange.MILLENNIUM,
+                "TIME_UNIT_RANGE",
+                "MILLENNIUM");
+

Review Comment:
   remove empty line here



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala:
##########
@@ -1801,6 +1801,12 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testExtract(): Unit = {

Review Comment:
   We don't need to add tests to the old test base. Can we remove this method entirely because `ExtractFunctionITCase` is much nicer.



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala:
##########
@@ -168,6 +168,12 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
     testSqlApi("EXTRACT(ISOYEAR FROM TIME '12:42:25')", "0")
   }
 
+  @Test
+  def testEpochWithTimeWhichIsUnsupported(): Unit = {

Review Comment:
   Let's not add new tests to outdated test bases. Use `ExtractFunctionITCase`.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/ExtractFunctionITCase.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.planner.functions;
+
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import java.time.LocalDateTime;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.api.DataTypes.BIGINT;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.call;
+
+/** Test {@link BuiltInFunctionDefinitions#EXTRACT} and its return type. */
+class ExtractFunctionITCase extends BuiltInFunctionTestBase {
+    @Override
+    Stream<TestSetSpec> getTestSetSpecs() {
+        return Stream.of(
+                TestSetSpec.forFunction(BuiltInFunctionDefinitions.EXTRACT)
+                        .onFieldsWithData(
+                                LocalDateTime.of(2000, 1, 31, 11, 22, 33, 123456789),
+                                LocalDateTime.of(2020, 2, 29, 1, 56, 59, 987654321),

Review Comment:
   What is the current behavior of `EXTRACT` on `TIMESTAMP_LTZ`? Can we add a test for it? At least for testing the error behavior?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/ExtractFunctionITCase.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.planner.functions;
+
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import java.time.LocalDateTime;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.api.DataTypes.BIGINT;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.call;
+
+/** Test {@link BuiltInFunctionDefinitions#EXTRACT} and its return type. */
+class ExtractFunctionITCase extends BuiltInFunctionTestBase {
+    @Override
+    Stream<TestSetSpec> getTestSetSpecs() {
+        return Stream.of(
+                TestSetSpec.forFunction(BuiltInFunctionDefinitions.EXTRACT)
+                        .onFieldsWithData(
+                                LocalDateTime.of(2000, 1, 31, 11, 22, 33, 123456789),

Review Comment:
   Can you add a test for extracting `EPOCH` on `DATE`? As far as I see it in the code, we also support it now?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/ExtractFunctionITCase.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.planner.functions;
+
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import java.time.LocalDateTime;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.api.DataTypes.BIGINT;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.call;
+
+/** Test {@link BuiltInFunctionDefinitions#EXTRACT} and its return type. */
+class ExtractFunctionITCase extends BuiltInFunctionTestBase {
+    @Override
+    Stream<TestSetSpec> getTestSetSpecs() {
+        return Stream.of(
+                TestSetSpec.forFunction(BuiltInFunctionDefinitions.EXTRACT)
+                        .onFieldsWithData(
+                                LocalDateTime.of(2000, 1, 31, 11, 22, 33, 123456789),
+                                LocalDateTime.of(2020, 2, 29, 1, 56, 59, 987654321),
+                                null)
+                        .andDataTypes(
+                                TIMESTAMP().nullable(),
+                                TIMESTAMP().nullable(),
+                                TIMESTAMP().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.NANOSECOND, $("f0")),
+                                "EXTRACT(NANOSECOND FROM f0)",
+                                123456000L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.NANOSECOND, $("f1")),
+                                "EXTRACT(NANOSECOND FROM f1)",
+                                987654000L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.NANOSECOND, $("f2")),
+                                "EXTRACT(NANOSECOND FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MICROSECOND, $("f0")),
+                                "EXTRACT(MICROSECOND FROM f0)",
+                                123456L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MICROSECOND, $("f1")),
+                                "EXTRACT(MICROSECOND FROM f1)",
+                                987654L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MICROSECOND, $("f2")),
+                                "EXTRACT(MICROSECOND FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MILLISECOND, $("f0")),
+                                "EXTRACT(MILLISECOND FROM f0)",
+                                123L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MILLISECOND, $("f1")),
+                                "EXTRACT(MILLISECOND FROM f1)",
+                                987L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MILLISECOND, $("f2")),
+                                "EXTRACT(MILLISECOND FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.SECOND, $("f0")),
+                                "EXTRACT(SECOND FROM f0)",
+                                33L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.SECOND, $("f1")),
+                                "EXTRACT(SECOND FROM f1)",
+                                59L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.SECOND, $("f2")),
+                                "EXTRACT(SECOND FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MINUTE, $("f0")),
+                                "EXTRACT(MINUTE FROM f0)",
+                                22L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MINUTE, $("f1")),
+                                "EXTRACT(MINUTE FROM f1)",
+                                56L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MINUTE, $("f2")),
+                                "EXTRACT(MINUTE FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.HOUR, $("f0")),
+                                "EXTRACT(HOUR FROM f0)",
+                                11L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.HOUR, $("f1")),
+                                "EXTRACT(HOUR FROM f1)",
+                                1L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.HOUR, $("f2")),
+                                "EXTRACT(HOUR FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DAY, $("f0")),
+                                "EXTRACT(DAY FROM f0)",
+                                31L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DAY, $("f1")),
+                                "EXTRACT(DAY FROM f1)",
+                                29L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DAY, $("f2")),
+                                "EXTRACT(DAY FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.WEEK, $("f0")),
+                                "EXTRACT(WEEK FROM f0)",
+                                5L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.WEEK, $("f1")),
+                                "EXTRACT(WEEK FROM f1)",
+                                9L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.WEEK, $("f2")),
+                                "EXTRACT(WEEK FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MONTH, $("f0")),
+                                "EXTRACT(MONTH FROM f0)",
+                                1L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MONTH, $("f1")),
+                                "EXTRACT(MONTH FROM f1)",
+                                2L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MONTH, $("f2")),
+                                "EXTRACT(MONTH FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.QUARTER, $("f0")),
+                                "EXTRACT(QUARTER FROM f0)",
+                                1L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.QUARTER, $("f1")),
+                                "EXTRACT(QUARTER FROM f1)",
+                                1L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.QUARTER, $("f2")),
+                                "EXTRACT(QUARTER FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.YEAR, $("f0")),
+                                "EXTRACT(YEAR FROM f0)",
+                                2000L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.YEAR, $("f1")),
+                                "EXTRACT(YEAR FROM f1)",
+                                2020L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.YEAR, $("f2")),
+                                "EXTRACT(YEAR FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DECADE, $("f0")),
+                                "EXTRACT(DECADE FROM f0)",
+                                200L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DECADE, $("f1")),
+                                "EXTRACT(DECADE FROM f1)",
+                                202L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.DECADE, $("f2")),
+                                "EXTRACT(DECADE FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.CENTURY, $("f0")),
+                                "EXTRACT(CENTURY FROM f0)",
+                                20L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.CENTURY, $("f1")),
+                                "EXTRACT(CENTURY FROM f1)",
+                                21L,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.CENTURY, $("f2")),
+                                "EXTRACT(CENTURY FROM f2)",
+                                null,
+                                BIGINT().nullable())
+                        .testResult(
+                                call("EXTRACT", TimeIntervalUnit.MILLENNIUM, $("f0")),

Review Comment:
   use `$("f0").extract(...)` for Table API everywhere, instead of lookup via name



##########
docs/content/docs/dev/table/functions/systemFunctions.md:
##########
@@ -146,6 +146,7 @@ For Table API, please use `_` for spaces (e.g., `DAY_TO_HOUR`).
 |                          | `MICROSECOND`                  |
 | `DOY` _(SQL-only)_       |                                |
 | `DOW` _(SQL-only)_       |                                |
+| `EPOCH` _(SQL-only)_     |                                |

Review Comment:
   You updated `TimeIntervalUnit.java`. So many symbols in this table are not `SQL-only` anymore?



-- 
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