Airblader commented on a change in pull request #16020:
URL: https://github.com/apache/flink/pull/16020#discussion_r670563688



##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
##########
@@ -283,6 +283,26 @@
                     
.outputTypeStrategy(nullable(explicit(DataTypes.BOOLEAN())))
                     .build();
 
+    public static final BuiltInFunctionDefinition GREATEST =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("greatest")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            comparable(ConstantArgumentCount.from(1), 
StructuredComparision.FULL))
+                    .outputTypeStrategy(nullable(TypeStrategies.COMMON))

Review comment:
       Should we try to make the output type strategy more narrow by only 
making it nullable if any of the provided input arguments is nullable?

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
##########
@@ -283,6 +283,26 @@
                     
.outputTypeStrategy(nullable(explicit(DataTypes.BOOLEAN())))
                     .build();
 
+    public static final BuiltInFunctionDefinition GREATEST =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("greatest")

Review comment:
       nit: we're trying to migrate everything to uppercase for built-in 
functions, so maybe for the new ones we can use it already?

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
##########
@@ -1767,6 +1767,56 @@ object ScalarOperatorGens {
     }
   }
 
+  /**
+   * Return null when at least one of the elements is null.
+   */
+  def generateGreatestLeast(
+      resultType: LogicalType,
+      elements: Seq[GeneratedExpression],
+      greatest: Boolean = true)
+  : GeneratedExpression = {
+    val Seq(result, cur, nullTerm) = newNames("result", "cur", "nullTerm")
+    val widerType = toScala(findCommonType(elements.map(element => 
element.resultType)))
+      .orElse(throw new CodeGenException(s"Unable to find common type for 
$elements."))
+    val resultTypeTerm = boxedTypeTermForType(widerType.get)
+
+    def castIfNumeric(t: GeneratedExpression): String = {
+      if (TypeCheckUtils.isNumeric(widerType.get)) {
+         s"${numericCasting(t.resultType, widerType.get).apply(t.resultTerm)}"
+      } else {
+         s"${t.resultTerm}"
+      }
+    }
+
+    val elementsCode = elements.map { element =>
+      s"""
+         | ${element.code}
+         | if (!$nullTerm) {
+         |   $resultTypeTerm $cur = ${castIfNumeric(element)};
+         |   if (${element.nullTerm}) {
+         |     $nullTerm = true;
+         |   } else {
+         |     int compareResult = $result.compareTo($cur);
+         |     if ($greatest && compareResult < 0 || compareResult > 0 && 
!$greatest) {

Review comment:
       nit: this is generated code, so it's semi-important, but I'd find this a 
bit more readable:
   
   ```
   if (($greatest && compareResult < 0) || (!$greatest && compareResult > 0)) {
   ```

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
##########
@@ -1767,6 +1767,56 @@ object ScalarOperatorGens {
     }
   }
 
+  /**
+   * Return null when at least one of the elements is null.

Review comment:
       This explains only an edge case of the function generated here, but not 
really what is being generated. I think we should properly document what this 
method does.

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
##########
@@ -283,6 +283,26 @@
                     
.outputTypeStrategy(nullable(explicit(DataTypes.BOOLEAN())))
                     .build();
 
+    public static final BuiltInFunctionDefinition GREATEST =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("greatest")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            comparable(ConstantArgumentCount.from(1), 
StructuredComparision.FULL))
+                    .outputTypeStrategy(nullable(TypeStrategies.COMMON))
+                    .runtimeProvided()
+                    .build();
+
+    public static final BuiltInFunctionDefinition LEAST =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("least")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            comparable(ConstantArgumentCount.from(1), 
StructuredComparision.FULL))

Review comment:
       I'm just noticing that `StructuredComparision` has a typo (should be 
`StructuredComparison`). This isn't part of your PR, but maybe you can add a 
hotfix commit to fix it.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to