You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ma...@apache.org on 2022/03/30 13:44:01 UTC
[flink] branch release-1.15 updated: [FLINK-25227][table-planner] Fix LEAST/GREATEST to return primitives
This is an automated email from the ASF dual-hosted git repository.
martijnvisser pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.15 by this push:
new e0c8df1 [FLINK-25227][table-planner] Fix LEAST/GREATEST to return primitives
e0c8df1 is described below
commit e0c8df17f3b0469affeb4c07be5b7c03d3a94333
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Tue Mar 29 17:31:11 2022 +0300
[FLINK-25227][table-planner] Fix LEAST/GREATEST to return primitives
Previously, `LEAST` and `GREATEST` functions would return primitive
types in the generated code implementing their logic, producing issues
for operators applied on top of them, and most importantly comparison
operators, i.e.:
```
f0 INT, f1 INT
SELECT GREATEST(f0, f1) = GREATEST(f0, f1)
```
would return `FALSE`, since the generated code would return `Integer`
instead of `int`, as the result of `GREATEST`, and the `=` operator
on `Integer` objects would return false, even if the actual integer
value of them was the same.
(cherry picked from commit 89cdc6e01f291e9ce5c1dfeb4bd883809e1eeaf5)
---
.../planner/codegen/calls/ScalarOperatorGens.scala | 18 ++---
.../functions/GreatestLeastFunctionsITCase.java | 83 ++++++++++++++++++++--
2 files changed, 89 insertions(+), 12 deletions(-)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
index 6288b96..6d920b4 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
@@ -1381,10 +1381,11 @@ object ScalarOperatorGens {
elements: Seq[GeneratedExpression],
greatest: Boolean = true)
: GeneratedExpression = {
- val Seq(result, cur, nullTerm) = newNames("result", "cur", "nullTerm")
+ val Seq(result, tmpResult, cur, nullTerm) = newNames("result", "tmpResult", "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)
+ val boxedResultTypeTerm = boxedTypeTermForType(widerType.get)
+ val primitiveResultTypeTerm = primitiveTypeTermForType(widerType.get)
def castIfNumeric(t: GeneratedExpression): String = {
if (isNumeric(widerType.get)) {
@@ -1398,13 +1399,13 @@ object ScalarOperatorGens {
s"""
| ${element.code}
| if (!$nullTerm) {
- | $resultTypeTerm $cur = ${castIfNumeric(element)};
+ | $boxedResultTypeTerm $cur = ${castIfNumeric(element)};
| if (${element.nullTerm}) {
| $nullTerm = true;
| } else {
- | int compareResult = $result.compareTo($cur);
+ | int compareResult = $tmpResult.compareTo($cur);
| if (($greatest && compareResult < 0) || (compareResult > 0 && !$greatest)) {
- | $result = $cur;
+ | $tmpResult = $cur;
| }
| }
| }
@@ -1413,11 +1414,12 @@ object ScalarOperatorGens {
val code =
s"""
- | $resultTypeTerm $result = ${castIfNumeric(elements.head)};
+ | $boxedResultTypeTerm $tmpResult = ${castIfNumeric(elements.head)};
+ | $primitiveResultTypeTerm $result = ${primitiveDefaultValue(widerType.get)};
| boolean $nullTerm = false;
| $elementsCode
- | if ($nullTerm) {
- | $result = null;
+ | if (!$nullTerm) {
+ | $result = $tmpResult;
| }
""".stripMargin
GeneratedExpression(result, nullTerm, code, resultType)
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java
index 1780438..d495c3f 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java
@@ -81,20 +81,63 @@ public class GreatestLeastFunctionsITCase extends BuiltInFunctionTestBase {
call("GREATEST", $("f6"), $("f7")),
"GREATEST(f6, f7)",
LocalDateTime.parse("1970-01-01T00:00:03.001"),
- DataTypes.TIMESTAMP(3).notNull()))
+ DataTypes.TIMESTAMP(3).notNull()),
+ resultSpec(
+ call("GREATEST", $("f0"), $("f1"), $("f2")),
+ "GREATEST(f0, f1, f2)",
+ null,
+ DataTypes.INT()),
+ resultSpec(
+ call("GREATEST", $("f4"), $("f5")),
+ "GREATEST(f4, f5)",
+ "world",
+ DataTypes.STRING().notNull()),
+ resultSpec(
+ call("GREATEST", $("f6"), $("f7")),
+ "GREATEST(f6, f7)",
+ LocalDateTime.parse("1970-01-01T00:00:03.001"),
+ DataTypes.TIMESTAMP(3).notNull()),
+ // assert that primitive types are returned and used in the equality
+ // operator applied on top of the GREATEST functions
+ resultSpec(
+ call(
+ "EQUALS",
+ call("GREATEST", $("f1"), $("f2")),
+ call("GREATEST", $("f1"), $("f2"))),
+ "GREATEST(f1, f2) = GREATEST(f1, f2)",
+ true,
+ DataTypes.BOOLEAN().notNull()),
+ resultSpec(
+ call(
+ "EQUALS",
+ call("GREATEST", $("f0"), $("f1")),
+ call("GREATEST", $("f0"), $("f1"))),
+ "GREATEST(f0, f1) = GREATEST(f0, f1)",
+ null,
+ DataTypes.BOOLEAN()))
.testSqlValidationError(
"GREATEST(f5, f6)",
"SQL validation failed. Invalid function call:\n"
+ "GREATEST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)"),
TestSpec.forFunction(BuiltInFunctionDefinitions.LEAST)
- .onFieldsWithData(null, 1, 2, 3.14, "hello", "world")
+ .onFieldsWithData(
+ null,
+ 1,
+ 2,
+ 3.14,
+ "hello",
+ "world",
+ LocalDateTime.parse("1970-01-01T00:00:03.001"),
+ LocalDateTime.parse("1970-01-01T00:00:02.001"))
.andDataTypes(
DataTypes.INT().nullable(),
DataTypes.INT().notNull(),
DataTypes.INT().notNull(),
DataTypes.DECIMAL(3, 2).notNull(),
DataTypes.STRING().notNull(),
- DataTypes.STRING().notNull())
+ DataTypes.STRING().notNull(),
+ DataTypes.TIMESTAMP(3).notNull(),
+ DataTypes.TIMESTAMP(3).notNull())
.testSqlValidationError(
"LEAST(f1, f4)",
"SQL validation failed. Invalid function call:\n"
@@ -115,6 +158,38 @@ public class GreatestLeastFunctionsITCase extends BuiltInFunctionTestBase {
call("LEAST", $("f4"), $("f5")),
"LEAST(f4, f5)",
"hello",
- DataTypes.STRING().notNull())));
+ DataTypes.STRING().notNull()),
+ resultSpec(
+ call("LEAST", $("f0"), $("f1")),
+ "LEAST(f0, f1)",
+ null,
+ DataTypes.INT()),
+ resultSpec(
+ call("LEAST", $("f4"), $("f5")),
+ "LEAST(f4, f5)",
+ "hello",
+ DataTypes.STRING().notNull()),
+ // assert that primitive types are returned and used in the equality
+ // operator applied on top of the GREATEST functions
+ resultSpec(
+ call(
+ "EQUALS",
+ call("LEAST", $("f1"), $("f2")),
+ call("LEAST", $("f1"), $("f2"))),
+ "LEAST(f1, f2) = LEAST(f1, f2)",
+ true,
+ DataTypes.BOOLEAN().notNull()),
+ resultSpec(
+ call(
+ "EQUALS",
+ call("LEAST", $("f0"), $("f1")),
+ call("LEAST", $("f0"), $("f1"))),
+ "LEAST(f0, f1) = LEAST(f0, f1)",
+ null,
+ DataTypes.BOOLEAN()))
+ .testSqlValidationError(
+ "LEAST(f5, f6)",
+ "SQL validation failed. Invalid function call:\n"
+ + "LEAST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)"));
}
}