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/04 11:08:16 UTC

[GitHub] [flink] twalthr commented on a diff in pull request #19190: [FLINK-13785][table] Port time functions to new type inference

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


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/CallContext.java:
##########
@@ -93,6 +93,17 @@ default ValidationException newValidationError(String message, Object... args) {
         return new ValidationException(String.format(message, args));
     }
 
+    /**
+     * Shorthand for {@code if (throwOnFailure) throw ValidationException(...) else return
+     * Optional.empty()}.
+     */
+    default <T> Optional<T> failInference(boolean throwOnFailure, String message, Object... args) {

Review Comment:
   can we give a more meaningful name here? maybe `emptyOrError`? It would read nicely `return emptyOrError();`.



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java:
##########
@@ -1222,76 +1231,103 @@
             BuiltInFunctionDefinition.newBuilder()
                     .name("extract")
                     .kind(SCALAR)
-                    .outputTypeStrategy(TypeStrategies.MISSING)
+                    .inputTypeStrategy(SpecificInputTypeStrategies.EXTRACT)
+                    .outputTypeStrategy(explicit(BIGINT().notNull()))

Review Comment:
   +1 to Marios comment



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/Signature.java:
##########
@@ -81,11 +85,52 @@ public static Argument of(String name, String type) {
             return new Argument(Preconditions.checkNotNull(name, "Name must not be null."), type);
         }
 
+        public static Argument of(String name, LogicalType type) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + type.asSummaryString() + ">");

Review Comment:
   remove `<...>` here, `"< ... >" to indicate that this is not a type`



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/Signature.java:
##########
@@ -81,11 +85,52 @@ public static Argument of(String name, String type) {
             return new Argument(Preconditions.checkNotNull(name, "Name must not be null."), type);
         }
 
+        public static Argument of(String name, LogicalType type) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + type.asSummaryString() + ">");
+        }
+
+        public static Argument of(String name, LogicalTypeRoot typeRoot) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + typeRoot + ">");
+        }
+
+        public static Argument of(String name, LogicalTypeFamily typeFamily) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + typeFamily + ">");
+        }
+
+        public static Argument of(
+                String name, Class<? extends Enum<? extends TableSymbol>> symbol) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + symbol.getSimpleName() + ">");
+        }
+
         /** Returns an instance of {@link Argument}. */
         public static Argument of(String type) {
             return new Argument(null, type);
         }
 
+        public static Argument of(LogicalType type) {
+            return new Argument(null, "<" + type.asSummaryString() + ">");
+        }
+
+        public static Argument of(LogicalTypeRoot typeRoot) {
+            return new Argument(null, "<" + typeRoot + ">");
+        }
+
+        public static Argument of(LogicalTypeFamily typeFamily) {
+            return new Argument(null, "<" + typeFamily + ">");
+        }
+
+        public static Argument of(Class<? extends Enum<?>> symbol) {

Review Comment:
   `extends TableSymbol`



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ExtractInputTypeStrategy.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.types.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentCount;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.ConstantArgumentCount;
+import org.apache.flink.table.types.inference.InputTypeStrategy;
+import org.apache.flink.table.types.inference.Signature;
+import org.apache.flink.table.types.inference.Signature.Argument;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.utils.DateTimeUtils;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME;
+
+/**
+ * Type strategy for EXTRACT, checking the first value is a valid literal of type {@link
+ * TimeIntervalUnit}, and that the combination of the second argument type and the interval unit is
+ * correct.
+ */
+@Internal
+public class ExtractInputTypeStrategy implements InputTypeStrategy {

Review Comment:
   default scoped?



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/Signature.java:
##########
@@ -81,11 +85,52 @@ public static Argument of(String name, String type) {
             return new Argument(Preconditions.checkNotNull(name, "Name must not be null."), type);
         }
 
+        public static Argument of(String name, LogicalType type) {
+            return new Argument(
+                    Preconditions.checkNotNull(name, "Name must not be null."),
+                    "<" + type.asSummaryString() + ">");
+        }
+
+        public static Argument of(String name, LogicalTypeRoot typeRoot) {

Review Comment:
   let's offer a `ofSpecific(String name, Object toString)` that will be printed as `"<...>"`, we don't need the following `of` methods then



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/TemporalOverlapsInputTypeStrategy.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.types.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentCount;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.ConstantArgumentCount;
+import org.apache.flink.table.types.inference.InputTypeStrategy;
+import org.apache.flink.table.types.inference.Signature;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+
+/** Type strategy of {@code TO_TIMESTAMP_LTZ}. */
+@Internal
+public class TemporalOverlapsInputTypeStrategy implements InputTypeStrategy {

Review Comment:
   default scoped



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