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

dwysakowicz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 54906feae0b [FLINK-33546] Implement type inference for order(asc/desc) 
function
54906feae0b is described below

commit 54906feae0bb2978bbd58bf91c6d458d44262437
Author: Dawid Wysakowicz <[email protected]>
AuthorDate: Thu Nov 16 12:20:07 2023 +0100

    [FLINK-33546] Implement type inference for order(asc/desc) function
    
    This closes #23733
---
 .../functions/BuiltInFunctionDefinitions.java      |  6 ++-
 .../expressions/PlannerExpressionConverter.scala   |  8 ----
 .../flink/table/planner/expressions/ordering.scala | 43 ----------------------
 3 files changed, 4 insertions(+), 53 deletions(-)

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 6f34233e6d4..4b208552d73 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
@@ -2005,14 +2005,16 @@ public final class BuiltInFunctionDefinitions {
             BuiltInFunctionDefinition.newBuilder()
                     .name("asc")
                     .kind(OTHER)
-                    .outputTypeStrategy(TypeStrategies.MISSING)
+                    .inputTypeStrategy(sequence(ANY))
+                    .outputTypeStrategy(argument(0))
                     .build();
 
     public static final BuiltInFunctionDefinition ORDER_DESC =
             BuiltInFunctionDefinition.newBuilder()
                     .name("desc")
                     .kind(OTHER)
-                    .outputTypeStrategy(TypeStrategies.MISSING)
+                    .inputTypeStrategy(sequence(ANY))
+                    .outputTypeStrategy(argument(0))
                     .build();
 
     // 
--------------------------------------------------------------------------------------------
diff --git 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
index 9272f2946d7..5f3953bfdd7 100644
--- 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
+++ 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
@@ -117,14 +117,6 @@ class PlannerExpressionConverter private extends 
ApiExpressionVisitor[PlannerExp
       case fd: FunctionDefinition =>
         fd match {
 
-          case ORDER_ASC =>
-            assert(args.size == 1)
-            Asc(args.head)
-
-          case ORDER_DESC =>
-            assert(args.size == 1)
-            Desc(args.head)
-
           case OVER =>
             assert(args.size >= 4)
             OverCall(
diff --git 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/ordering.scala
 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/ordering.scala
deleted file mode 100644
index 956f8fb827e..00000000000
--- 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/ordering.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.planner.validate._
-
-abstract class Ordering extends UnaryExpression {
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!child.isInstanceOf[NamedExpression]) {
-      ValidationFailure(s"Sort should only based on field reference")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-case class Asc(child: PlannerExpression) extends Ordering {
-  override def toString: String = s"($child).asc"
-
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-}
-
-case class Desc(child: PlannerExpression) extends Ordering {
-  override def toString: String = s"($child).desc"
-
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-}

Reply via email to