You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/05 14:46:24 UTC
[flink] 05/05: [FLINK-13237][table-planner-blink] Add expression
table api tests in blink planner
This is an automated email from the ASF dual-hosted git repository.
jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 949b7e17479c954477717ba18fde8922ca6764f9
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Fri Jul 12 11:53:12 2019 +0800
[FLINK-13237][table-planner-blink] Add expression table api tests in blink planner
This closes #9099
---
.../table/planner/expressions/ArrayTypeTest.scala | 202 ++-
.../planner/expressions/CompositeAccessTest.scala | 85 +-
.../planner/expressions/DecimalTypeTest.scala | 199 ++-
.../table/planner/expressions/LiteralTest.scala | 82 +-
.../table/planner/expressions/MapTypeTest.scala | 165 +-
.../planner/expressions/MathFunctionsTest.scala | 694 --------
.../expressions/NonDeterministicTests.scala | 32 +-
.../table/planner/expressions/RowTypeTest.scala | 63 +-
.../planner/expressions/ScalarFunctionsTest.scala | 1879 +++++++++++++++-----
.../planner/expressions/TemporalTypesTest.scala | 357 +++-
.../UserDefinedScalarFunctionTest.scala | 485 +++++
.../expressions/utils/ExpressionTestBase.scala | 71 +-
.../utils/ScalarOperatorsTestBase.scala | 5 +
.../utils/userDefinedScalarFunctions.scala | 28 +
.../validation/ArrayTypeValidationTest.scala | 43 +
.../validation/CompositeAccessValidationTest.scala | 21 +
.../validation/MapTypeValidationTest.scala | 10 +-
.../validation/RowTypeValidationTest.scala | 5 +-
.../validation/ScalarFunctionsValidationTest.scala | 112 +-
.../validation/ScalarOperatorsValidationTest.scala | 88 +
20 files changed, 3202 insertions(+), 1424 deletions(-)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
index 82c9229..4e3acd7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
@@ -18,7 +18,10 @@
package org.apache.flink.table.planner.expressions
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
import org.junit.Test
@@ -27,115 +30,175 @@ class ArrayTypeTest extends ArrayTypeTestBase {
@Test
def testArrayLiterals(): Unit = {
// primitive literals
- testSqlApi(
- "ARRAY[1, 2, 3]",
- "[1, 2, 3]")
+ testAllApis(array(1, 2, 3), "array(1, 2, 3)", "ARRAY[1, 2, 3]", "[1, 2, 3]")
- testSqlApi(
+ testAllApis(
+ array(true, true, true),
+ "array(true, true, true)",
"ARRAY[TRUE, TRUE, TRUE]",
"[true, true, true]")
- testSqlApi(
+ // object literals
+ testTableApi(array(BigDecimal(1), BigDecimal(1)), "array(1p, 1p)", "[1, 1]")
+
+ testAllApis(
+ array(array(array(1), array(1))),
+ "array(array(array(1), array(1)))",
"ARRAY[ARRAY[ARRAY[1], ARRAY[1]]]",
"[[[1], [1]]]")
- testSqlApi(
+ testAllApis(
+ array(1 + 1, 3 * 3),
+ "array(1 + 1, 3 * 3)",
"ARRAY[1 + 1, 3 * 3]",
"[2, 9]")
- testSqlApi(
+ testAllApis(
+ array(nullOf(DataTypes.INT), 1),
+ "array(Null(INT), 1)",
"ARRAY[NULLIF(1,1), 1]",
"[null, 1]")
- testSqlApi(
+ testAllApis(
+ array(array(nullOf(DataTypes.INT), 1)),
+ "array(array(Null(INT), 1))",
"ARRAY[ARRAY[NULLIF(1,1), 1]]",
"[[null, 1]]")
- testSqlApi(
+ // implicit conversion
+ testTableApi(
+ Array(1, 2, 3),
+ "array(1, 2, 3)",
+ "[1, 2, 3]")
+
+ testTableApi(
+ Array[Integer](1, 2, 3),
+ "array(1, 2, 3)",
+ "[1, 2, 3]")
+
+ testAllApis(
+ Array(localDate("1985-04-11"), localDate("2018-07-26")),
+ "array('1985-04-11'.toDate, '2018-07-26'.toDate)",
"ARRAY[DATE '1985-04-11', DATE '2018-07-26']",
"[1985-04-11, 2018-07-26]")
- testSqlApi(
+ testAllApis(
+ Array(gLocalTime("14:15:16"), gLocalTime("17:18:19")),
+ "array('14:15:16'.toTime, '17:18:19'.toTime)",
"ARRAY[TIME '14:15:16', TIME '17:18:19']",
"[14:15:16, 17:18:19]")
- testSqlApi(
+ testAllApis(
+ Array(localDateTime("1985-04-11 14:15:16"), localDateTime("2018-07-26 17:18:19")),
+ "array('1985-04-11 14:15:16'.toTimestamp, '2018-07-26 17:18:19'.toTimestamp)",
"ARRAY[TIMESTAMP '1985-04-11 14:15:16', TIMESTAMP '2018-07-26 17:18:19']",
"[1985-04-11 14:15:16.000, 2018-07-26 17:18:19.000]")
- testSqlApi(
+ testAllApis(
+ Array(BigDecimal(2.0002), BigDecimal(2.0003)),
+ "Array(2.0002p, 2.0003p)",
"ARRAY[CAST(2.0002 AS DECIMAL(10,4)), CAST(2.0003 AS DECIMAL(10,4))]",
"[2.0002, 2.0003]")
- testSqlApi(
+ testAllApis(
+ Array(Array(x = true)),
+ "Array(Array(true))",
"ARRAY[ARRAY[TRUE]]",
"[[true]]")
- testSqlApi(
+ testAllApis(
+ Array(Array(1, 2, 3), Array(3, 2, 1)),
+ "Array(Array(1, 2, 3), Array(3, 2, 1))",
"ARRAY[ARRAY[1, 2, 3], ARRAY[3, 2, 1]]",
"[[1, 2, 3], [3, 2, 1]]")
// implicit type cast only works on SQL APIs.
- testSqlApi(
- "ARRAY[CAST(1 AS DOUBLE), CAST(2 AS FLOAT)]",
- "[1.0, 2.0]")
+ testSqlApi("ARRAY[CAST(1 AS DOUBLE), CAST(2 AS FLOAT)]", "[1.0, 2.0]")
}
@Test
def testArrayField(): Unit = {
- testSqlApi(
+ testAllApis(
+ array('f0, 'f1),
+ "array(f0, f1)",
"ARRAY[f0, f1]",
"[null, 42]")
- testSqlApi(
+ testAllApis(
+ array('f0, 'f1),
+ "array(f0, f1)",
"ARRAY[f0, f1]",
"[null, 42]")
- testSqlApi(
+ testAllApis(
+ 'f2,
+ "f2",
"f2",
"[1, 2, 3]")
- testSqlApi(
+ testAllApis(
+ 'f3,
+ "f3",
"f3",
"[1984-03-12, 1984-02-10]")
- testSqlApi(
+ testAllApis(
+ 'f5,
+ "f5",
"f5",
"[[1, 2, 3], null]")
- testSqlApi(
+ testAllApis(
+ 'f6,
+ "f6",
"f6",
"[1, null, null, 4]")
- testSqlApi(
+ testAllApis(
+ 'f2,
+ "f2",
"f2",
"[1, 2, 3]")
- testSqlApi(
+ testAllApis(
+ 'f2.at(1),
+ "f2.at(1)",
"f2[1]",
"1")
- testSqlApi(
+ testAllApis(
+ 'f3.at(1),
+ "f3.at(1)",
"f3[1]",
"1984-03-12")
- testSqlApi(
+ testAllApis(
+ 'f3.at(2),
+ "f3.at(2)",
"f3[2]",
"1984-02-10")
- testSqlApi(
+ testAllApis(
+ 'f5.at(1).at(2),
+ "f5.at(1).at(2)",
"f5[1][2]",
"2")
- testSqlApi(
+ testAllApis(
+ 'f5.at(2).at(2),
+ "f5.at(2).at(2)",
"f5[2][2]",
"null")
- testSqlApi(
+ testAllApis(
+ 'f4.at(2).at(2),
+ "f4.at(2).at(2)",
"f4[2][2]",
"null")
- testSqlApi(
+ testAllApis(
+ 'f11.at(1),
+ "f11.at(1)",
"f11[1]",
"1")
}
@@ -143,74 +206,117 @@ class ArrayTypeTest extends ArrayTypeTestBase {
@Test
def testArrayOperations(): Unit = {
// cardinality
- testSqlApi(
+ testAllApis(
+ 'f2.cardinality(),
+ "f2.cardinality()",
"CARDINALITY(f2)",
"3")
- testSqlApi(
+ testAllApis(
+ 'f4.cardinality(),
+ "f4.cardinality()",
"CARDINALITY(f4)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f11.cardinality(),
+ "f11.cardinality()",
"CARDINALITY(f11)",
"1")
// element
- testSqlApi(
+ testAllApis(
+ 'f9.element(),
+ "f9.element()",
"ELEMENT(f9)",
"1")
- testSqlApi(
+ testAllApis(
+ 'f8.element(),
+ "f8.element()",
"ELEMENT(f8)",
"4.0")
- testSqlApi(
+ testAllApis(
+ 'f10.element(),
+ "f10.element()",
"ELEMENT(f10)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f4.element(),
+ "f4.element()",
"ELEMENT(f4)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f11.element(),
+ "f11.element()",
"ELEMENT(f11)",
"1")
// comparison
- testSqlApi(
+ testAllApis(
+ 'f2 === 'f5.at(1),
+ "f2 === f5.at(1)",
"f2 = f5[1]",
"true")
- testSqlApi(
+ testAllApis(
+ 'f6 === array(1, 2, 3),
+ "f6 === array(1, 2, 3)",
"f6 = ARRAY[1, 2, 3]",
"false")
- testSqlApi(
+ testAllApis(
+ 'f2 !== 'f5.at(1),
+ "f2 !== f5.at(1)",
"f2 <> f5[1]",
"false")
- testSqlApi(
+ testAllApis(
+ 'f2 === 'f7,
+ "f2 === f7",
"f2 = f7",
"false")
- testSqlApi(
+ testAllApis(
+ 'f2 !== 'f7,
+ "f2 !== f7",
"f2 <> f7",
"true")
- testSqlApi(
+ testAllApis(
+ 'f11 === 'f11,
+ "f11 === f11",
"f11 = f11",
"true")
- testSqlApi(
+ testAllApis(
+ 'f11 === 'f9,
+ "f11 === f9",
"f11 = f9",
"true")
- testSqlApi(
+ testAllApis(
+ 'f11 !== 'f11,
+ "f11 !== f11",
"f11 <> f11",
"false")
- testSqlApi(
+ testAllApis(
+ 'f11 !== 'f9,
+ "f11 !== f9",
"f11 <> f9",
"false")
}
+
+ @Test
+ def testArrayTypeCasting(): Unit = {
+ testTableApi(
+ 'f3.cast(DataTypes.ARRAY(DataTypes.DATE)),
+ "f3.cast(OBJECT_ARRAY(SQL_DATE))",
+ "[1984-03-12, 1984-02-10]"
+ )
+ }
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
index 20e34a9..43b8160 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.expressions
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.CompositeTypeTestBase
import org.junit.Test
@@ -28,7 +29,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
def testGetField(): Unit = {
// single field by string key
- testSqlApi(
+ testAllApis(
+ 'f0.get("intField"),
+ "f0.get('intField')",
"testTable.f0.intField",
"42")
testSqlApi("f0.intField", "42")
@@ -39,8 +42,16 @@ class CompositeAccessTest extends CompositeTypeTestBase {
testSqlApi("testTable.f0.booleanField", "true")
testSqlApi("f0.booleanField", "true")
+ // single field by int key
+ testTableApi(
+ 'f0.get(0),
+ "f0.get(0)",
+ "42")
+
// nested single field
- testSqlApi(
+ testAllApis(
+ 'f1.get("objectField").get("intField"),
+ "f1.get('objectField').get('intField')",
"testTable.f1.objectField.intField",
"25")
testSqlApi("f1.objectField.intField", "25")
@@ -51,7 +62,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
testSqlApi("testTable.f1.objectField.booleanField", "false")
testSqlApi("f1.objectField.booleanField", "false")
- testSqlApi(
+ testAllApis(
+ 'f2.get(0),
+ "f2.get(0)",
"testTable.f2._1",
"a")
testSqlApi("f2._1", "a")
@@ -65,7 +78,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
testSqlApi("testTable.f5", "13")
testSqlApi("f5", "13")
- testSqlApi(
+ testAllApis(
+ 'f7.get("_1"),
+ "get(f7, '_1')",
"testTable.f7._1",
"true")
@@ -73,70 +88,82 @@ class CompositeAccessTest extends CompositeTypeTestBase {
testSqlApi("testTable.f6", "MyCaseClass2(null)")
testSqlApi("f6", "MyCaseClass2(null)")
- // MyCaseClass is converted to BaseRow
- // so the result of "toString" does'nt contain MyCaseClass prefix
- testSqlApi(
+ testAllApis(
+ 'f1.get("objectField"),
+ "f1.get('objectField')",
"testTable.f1.objectField",
"(25,Timo,false)")
testSqlApi("f1.objectField", "(25,Timo,false)")
- testSqlApi(
+ testAllApis(
+ 'f0,
+ "f0",
"testTable.f0",
"(42,Bob,true)")
testSqlApi("f0", "(42,Bob,true)")
// flattening (test base only returns first column)
- testSqlApi(
+ testAllApis(
+ 'f1.get("objectField").flatten(),
+ "f1.get('objectField').flatten()",
"testTable.f1.objectField.*",
"25")
testSqlApi("f1.objectField.*", "25")
- testSqlApi(
+ testAllApis(
+ 'f0.flatten(),
+ "flatten(f0)",
"testTable.f0.*",
"42")
testSqlApi("f0.*", "42")
+ testTableApi(12.flatten(), "12.flatten()", "12")
+
+ testTableApi('f5.flatten(), "f5.flatten()", "13")
+
// array of composites
- testSqlApi(
+ testAllApis(
+ 'f8.at(1).get("_1"),
+ "f8.at(1).get('_1')",
"f8[1]._1",
"true"
)
-
- testSqlApi(
+ testAllApis(
+ 'f8.at(1).get("_2"),
+ "f8.at(1).get('_2')",
"f8[1]._2",
"23"
)
-
- testSqlApi(
+ testAllApis(
+ 'f9.at(2).get("_1"),
+ "f9.at(2).get('_1')",
"f9[2]._1",
"null"
)
-
- testSqlApi(
+ testAllApis(
+ 'f10.at(1).get("stringField"),
+ "f10.at(1).get('stringField')",
"f10[1].stringField",
"Bob"
)
-
- testSqlApi(
+ testAllApis(
+ 'f11.at(1).get("myString"),
+ "f11.at(1).get('myString')",
"f11[1].myString",
"Hello"
)
-
- testSqlApi(
- "f11[2]",
- "null"
- )
-
- testSqlApi(
+ testAllApis(
+ 'f12.at(1).get("arrayField").at(1).get("stringField"),
+ "f12.at(1).get('arrayField').at(1).get('stringField')",
"f12[1].arrayField[1].stringField",
"Alice"
)
- testSqlApi(
+ testAllApis(
+ 'f13.at(1).get("objectField").get("stringField"),
+ "f13.at(1).get('objectField').get('stringField')",
"f13[1].objectField.stringField",
"Bob"
)
}
}
-
-
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
index b19c4d2..e243930 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
@@ -18,101 +18,165 @@
package org.apache.flink.table.planner.expressions
-import org.apache.flink.api.common.typeinfo.Types
import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.dataformat.Decimal
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{DataTypes, Types}
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral
import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo
import org.apache.flink.types.Row
-import org.junit.Test
+import org.junit.{Ignore, Test}
class DecimalTypeTest extends ExpressionTestBase {
@Test
def testDecimalLiterals(): Unit = {
// implicit double
- testSqlApi(
+ testAllApis(
+ 11.2,
+ "11.2",
"11.2",
"11.2")
// implicit double
- testSqlApi(
+ testAllApis(
+ 0.7623533651719233,
+ "0.7623533651719233",
"0.7623533651719233",
"0.7623533651719233")
// explicit decimal (with precision of 19)
- testSqlApi(
+ testAllApis(
+ BigDecimal("1234567891234567891"),
+ "1234567891234567891p",
"1234567891234567891",
"1234567891234567891")
+
+ // explicit decimal (high precision, not SQL compliant)
+ testTableApi(
+ BigDecimal("123456789123456789123456789"),
+ "123456789123456789123456789p",
+ "123456789123456789123456789")
+
+ // explicit decimal (high precision, not SQL compliant)
+ testTableApi(
+ BigDecimal("12.3456789123456789123456789"),
+ "12.3456789123456789123456789p",
+ "12.3456789123456789123456789")
}
@Test
def testDecimalBorders(): Unit = {
- testSqlApi(
+ testAllApis(
+ Double.MaxValue,
+ Double.MaxValue.toString,
Double.MaxValue.toString,
Double.MaxValue.toString)
- testSqlApi(
+ testAllApis(
+ Double.MinValue,
+ Double.MinValue.toString,
Double.MinValue.toString,
Double.MinValue.toString)
- testSqlApi(
+ testAllApis(
+ Double.MinValue.cast(DataTypes.FLOAT),
+ s"${Double.MinValue}.cast(FLOAT)",
s"CAST(${Double.MinValue} AS FLOAT)",
Float.NegativeInfinity.toString)
- testSqlApi(
+ testAllApis(
+ Byte.MinValue.cast(DataTypes.TINYINT),
+ s"(${Byte.MinValue}).cast(BYTE)",
s"CAST(${Byte.MinValue} AS TINYINT)",
Byte.MinValue.toString)
- testSqlApi(
+ testAllApis(
+ Byte.MinValue.cast(DataTypes.TINYINT) - 1.cast(DataTypes.TINYINT),
+ s"(${Byte.MinValue}).cast(BYTE) - (1).cast(BYTE)",
s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
Byte.MaxValue.toString)
- testSqlApi(
+ testAllApis(
+ Short.MinValue.cast(DataTypes.SMALLINT),
+ s"(${Short.MinValue}).cast(SHORT)",
s"CAST(${Short.MinValue} AS SMALLINT)",
Short.MinValue.toString)
- testSqlApi(
+ testAllApis(
+ Int.MinValue.cast(DataTypes.INT) - 1,
+ s"(${Int.MinValue}).cast(INT) - 1",
s"CAST(${Int.MinValue} AS INT) - 1",
Int.MaxValue.toString)
- testSqlApi(
+ testAllApis(
+ Long.MinValue.cast(DataTypes.BIGINT()),
+ s"(${Long.MinValue}L).cast(LONG)",
s"CAST(${Long.MinValue} AS BIGINT)",
Long.MinValue.toString)
}
+ @Ignore
@Test
- def testDecimalCasting(): Unit = {
+ def testDefaultDecimalCasting(): Unit = {
// from String
- testSqlApi(
- "CAST('123456789123456789123456789' AS DECIMAL(27, 0))",
+ testTableApi(
+ "123456789123456789123456789".cast(DataTypes.DECIMAL(38, 0)),
+ "'123456789123456789123456789'.cast(DECIMAL)",
"123456789123456789123456789")
// from double
- testSqlApi(
+ testAllApis(
+ 'f3.cast(DataTypes.DECIMAL(38, 0)),
+ "f3.cast(DECIMAL)",
"CAST(f3 AS DECIMAL)",
"4")
+ }
+ @Test
+ def testDecimalCasting(): Unit = {
testSqlApi(
"CAST(f3 AS DECIMAL(10,2))",
"4.20"
)
// to double
- testSqlApi(
+ testAllApis(
+ 'f0.cast(DataTypes.DOUBLE),
+ "f0.cast(DOUBLE)",
"CAST(f0 AS DOUBLE)",
"1.2345678912345679E8")
// to int
- testSqlApi(
+ testAllApis(
+ 'f4.cast(DataTypes.INT),
+ "f4.cast(INT)",
"CAST(f4 AS INT)",
"123456789")
// to long
- testSqlApi(
+ testAllApis(
+ 'f4.cast(DataTypes.BIGINT()),
+ "f4.cast(LONG)",
"CAST(f4 AS BIGINT)",
"123456789")
+
+ // to boolean (not SQL compliant)
+ testTableApi(
+ 'f1.cast(DataTypes.BOOLEAN),
+ "f1.cast(BOOLEAN)",
+ "true")
+
+ testTableApi(
+ 'f5.cast(DataTypes.BOOLEAN),
+ "f5.cast(BOOLEAN)",
+ "false")
+
+ testTableApi(
+ BigDecimal("123456789.123456789123456789").cast(DataTypes.DOUBLE),
+ "(123456789.123456789123456789p).cast(DOUBLE)",
+ "1.2345678912345679E8")
}
@Test
@@ -123,72 +187,111 @@ class DecimalTypeTest extends ExpressionTestBase {
// Decimal+Double => Double.
// implicit cast to decimal
- testSqlApi(
+ testAllApis(
+ 'f1 + 12,
+ "f1 + 12",
"f1 + 12",
"123456789123456789123456801")
// implicit cast to decimal
- testSqlApi(
+ testAllApis(
+ valueLiteral(12) + 'f1,
+ "12 + f1",
"12 + f1",
"123456789123456789123456801")
- testSqlApi(
+ testAllApis(
+ 'f1 + BigDecimal("12.3"),
+ "f1 + 12.3p",
"f1 + 12.3",
"123456789123456789123456801.3"
)
- testSqlApi(
+ testAllApis(
+ valueLiteral(BigDecimal("12.3").bigDecimal) + 'f1,
+ "12.3p + f1",
"12.3 + f1",
"123456789123456789123456801.3")
- testSqlApi(
+ testAllApis(
+ 'f1 + 'f1,
+ "f1 + f1",
"f1 + f1",
"246913578246913578246913578")
- testSqlApi(
+ testAllApis(
+ 'f1 - 'f1,
+ "f1 - f1",
"f1 - f1",
"0")
- testSqlApi(
+ // exceeds max precision 38.
+ // 'f1 * 'f1,
+ // "f1 * f1",
+ // "f1 * f1",
+ // "15241578780673678546105778281054720515622620750190521")
+
+ testAllApis(
+ 'f1 / 'f1,
+ "f1 / f1",
"f1 / f1",
"1.00000000")
+ // Decimal(30,0) / Decimal(30, 0) => Decimal(61,31) => Decimal(38,8)
- testSqlApi(
+ testAllApis(
+ 'f1 % 'f1,
+ "f1 % f1",
"MOD(f1, f1)",
"0")
- testSqlApi(
+ testAllApis(
+ -'f0,
+ "-f0",
"-f0",
"-123456789.123456789123456789")
}
@Test
def testDecimalComparison(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f1 < 12,
+ "f1 < 12",
"f1 < 12",
"false")
- testSqlApi(
+ testAllApis(
+ 'f1 > 12,
+ "f1 > 12",
"f1 > 12",
"true")
- testSqlApi(
+ testAllApis(
+ 'f1 === 12,
+ "f1 === 12",
"f1 = 12",
"false")
- testSqlApi(
+ testAllApis(
+ 'f5 === 0,
+ "f5 === 0",
"f5 = 0",
"true")
- testSqlApi(
+ testAllApis(
+ 'f1 === BigDecimal("123456789123456789123456789"),
+ "f1 === 123456789123456789123456789p",
"f1 = CAST('123456789123456789123456789' AS DECIMAL(30, 0))",
"true")
- testSqlApi(
+ testAllApis(
+ 'f1 !== BigDecimal("123456789123456789123456789"),
+ "f1 !== 123456789123456789123456789p",
"f1 <> CAST('123456789123456789123456789' AS DECIMAL(30, 0))",
"false")
- testSqlApi(
+ testAllApis(
+ 'f4 < 'f0,
+ "f4 < f0",
"f4 < f0",
"true")
@@ -202,22 +305,22 @@ class DecimalTypeTest extends ExpressionTestBase {
override def testData: Row = {
val testData = new Row(6)
- testData.setField(0, Decimal.castFrom("123456789.123456789123456789", 30, 18))
- testData.setField(1, Decimal.castFrom("123456789123456789123456789", 30, 0))
+ testData.setField(0, BigDecimal("123456789.123456789123456789").bigDecimal)
+ testData.setField(1, BigDecimal("123456789123456789123456789").bigDecimal)
testData.setField(2, 42)
testData.setField(3, 4.2)
- testData.setField(4, Decimal.castFrom("123456789", 10, 0))
- testData.setField(5, Decimal.castFrom("0.000", 10, 3))
+ testData.setField(4, BigDecimal("123456789").bigDecimal)
+ testData.setField(5, BigDecimal("0.000").bigDecimal)
testData
}
override def typeInfo: RowTypeInfo = {
new RowTypeInfo(
- /* 0 */ DecimalTypeInfo.of(30, 18),
- /* 1 */ DecimalTypeInfo.of(30, 0),
- /* 2 */ Types.INT,
- /* 3 */ Types.DOUBLE,
- /* 4 */ DecimalTypeInfo.of(10, 0),
- /* 5 */ DecimalTypeInfo.of(10, 3))
+ /* 1 */ BigDecimalTypeInfo.of(30, 18),
+ /* 2 */ BigDecimalTypeInfo.of(30, 0),
+ /* 3 */ Types.INT,
+ /* 4 */ Types.DOUBLE,
+ /* 5 */ BigDecimalTypeInfo.of(10, 0),
+ /* 6 */ BigDecimalTypeInfo.of(10, 3))
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
index 4857648..5158b25 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
@@ -20,7 +20,9 @@ package org.apache.flink.table.planner.expressions
import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.planner.expressions.utils.{ExpressionTestBase, Func3}
import org.apache.flink.types.Row
import org.junit.Test
@@ -30,27 +32,37 @@ class LiteralTest extends ExpressionTestBase {
@Test
def testFieldWithBooleanPrefix(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'trUeX,
+ "trUeX",
"trUeX",
"trUeX_value"
)
- testSqlApi(
+ testAllApis(
+ 'FALSE_A,
+ "FALSE_A",
"FALSE_A",
"FALSE_A_value"
)
- testSqlApi(
+ testAllApis(
+ 'FALSE_AB,
+ "FALSE_AB",
"FALSE_AB",
"FALSE_AB_value"
)
- testSqlApi(
+ testAllApis(
+ true,
+ "trUe",
"trUe",
"true"
)
- testSqlApi(
+ testAllApis(
+ false,
+ "FALSE",
"FALSE",
"false"
)
@@ -58,11 +70,15 @@ class LiteralTest extends ExpressionTestBase {
@Test
def testNonAsciiLiteral(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f4.like("%测试%"),
+ "f4.like('%测试%')",
"f4 LIKE '%测试%'",
"true")
- testSqlApi(
+ testAllApis(
+ "Абвгде" + "谢谢",
+ "'Абвгде' + '谢谢'",
"'Абвгде' || '谢谢'",
"Абвгде谢谢")
}
@@ -70,9 +86,11 @@ class LiteralTest extends ExpressionTestBase {
@Test
def testDoubleQuote(): Unit = {
val hello = "\"<hello>\""
- testSqlApi(
- s"concat('a', ' ', '$hello')",
- s"a $hello")
+ testAllApis(
+ Func3(42, hello),
+ s"Func3(42, '$hello')",
+ s"Func3(42, '$hello')",
+ s"42 and $hello")
}
@Test
@@ -80,43 +98,57 @@ class LiteralTest extends ExpressionTestBase {
// these tests use Java/Scala escaping for non-quoting unicode characters
- testSqlApi(
+ testAllApis(
+ ">\n<",
+ "'>\n<'",
"'>\n<'",
">\n<")
- testSqlApi(
+ testAllApis(
+ ">\u263A<",
"'>\u263A<'",
- ">\u263A<")
-
- testSqlApi(
"'>\u263A<'",
">\u263A<")
- testSqlApi(
+ testAllApis(
+ ">\\<",
+ "'>\\<'",
"'>\\<'",
">\\<")
- testSqlApi(
+ testAllApis(
+ ">'<",
+ "'>''<'",
"'>''<'",
">'<")
- testSqlApi(
+ testAllApis(
+ " ",
+ "' '",
"' '",
" ")
- testSqlApi(
+ testAllApis(
+ "",
+ "''",
"''",
"")
- testSqlApi(
+ testAllApis(
+ ">foo([\\w]+)<",
+ "'>foo([\\w]+)<'",
"'>foo([\\w]+)<'",
">foo([\\w]+)<")
- testSqlApi(
+ testAllApis(
+ ">\\'\n<",
+ "\">\\'\n<\"",
"'>\\''\n<'",
">\\'\n<")
- testSqlApi(
+ testAllApis(
+ "It's me.",
+ "'It''s me.'",
"'It''s me.'",
"It's me.")
@@ -135,6 +167,10 @@ class LiteralTest extends ExpressionTestBase {
">\\\\<")
}
+ override def functions: Map[String, ScalarFunction] = Map(
+ "Func3" -> Func3
+ )
+
override def testData: Row = {
val testData = new Row(4)
testData.setField(0, "trUeX_value")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
index 3f45a58..5379e35 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
@@ -18,7 +18,11 @@
package org.apache.flink.table.planner.expressions
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral
import org.apache.flink.table.planner.expressions.utils.MapTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
import org.junit.Test
@@ -36,85 +40,121 @@ class MapTypeTest extends MapTypeTestBase {
@Test
def testMapLiteral(): Unit = {
// primitive literals
- testSqlApi(
- "MAP[1, 1]",
- "{1=1}")
+ testAllApis(map(1, 1), "map(1, 1)", "MAP[1, 1]", "{1=1}")
- testSqlApi(
+ testAllApis(
+ map(true, true),
+ "map(true, true)",
"map[TRUE, TRUE]",
"{true=true}")
- testSqlApi(
+ // object literals
+ testTableApi(map(BigDecimal(1), BigDecimal(1)), "map(1p, 1p)", "{1=1}")
+
+ testAllApis(
+ map(map(1, 2), map(3, 4)),
+ "map(map(1, 2), map(3, 4))",
"MAP[MAP[1, 2], MAP[3, 4]]",
"{{1=2}={3=4}}")
- testSqlApi(
+ testAllApis(
+ map(1 + 2, 3 * 3, 3 - 6, 4 - 2),
+ "map(1 + 2, 3 * 3, 3 - 6, 4 - 2)",
"map[1 + 2, 3 * 3, 3 - 6, 4 - 2]",
"{3=9, -3=2}")
- testSqlApi(
+ testAllApis(
+ map(1, nullOf(DataTypes.INT)),
+ "map(1, Null(INT))",
"map[1, NULLIF(1,1)]",
"{1=null}")
// explicit conversion
- testSqlApi(
+ testAllApis(
+ map(1, 2L , 3, 4L),
+ "map(1, 2L, 3, 4L)",
"MAP[1, CAST(2 AS BIGINT), 3, CAST(4 AS BIGINT)]",
"{1=2, 3=4}")
- testSqlApi(
+ testAllApis(
+ map(valueLiteral(localDate("1985-04-11")), valueLiteral(gLocalTime("14:15:16")),
+ valueLiteral(localDate("2018-07-26")), valueLiteral(gLocalTime("17:18:19"))),
+ "map('1985-04-11'.toDate, '14:15:16'.toTime, '2018-07-26'.toDate, '17:18:19'.toTime)",
"MAP[DATE '1985-04-11', TIME '14:15:16', DATE '2018-07-26', TIME '17:18:19']",
"{1985-04-11=14:15:16, 2018-07-26=17:18:19}")
- testSqlApi(
+ testAllApis(
+ map(valueLiteral(gLocalTime("14:15:16")), valueLiteral(localDateTime("1985-04-11 14:15:16")),
+ valueLiteral(gLocalTime("17:18:19")), valueLiteral(localDateTime("2018-07-26 17:18:19"))),
+ "map('14:15:16'.toTime, '1985-04-11 14:15:16'.toTimestamp, " +
+ "'17:18:19'.toTime, '2018-07-26 17:18:19'.toTimestamp)",
"MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
- "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']",
+ "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']",
"{14:15:16=1985-04-11 14:15:16.000, 17:18:19=2018-07-26 17:18:19.000}")
- testSqlApi(
+ testAllApis(
+ map(BigDecimal(2.0002), BigDecimal(2.0003)),
+ "map(2.0002p, 2.0003p)",
"MAP[CAST(2.0002 AS DECIMAL(5, 4)), CAST(2.0003 AS DECIMAL(5, 4))]",
"{2.0002=2.0003}")
// implicit type cast only works on SQL API
- testSqlApi(
- "MAP['k1', CAST(1 AS DOUBLE), 'k2', CAST(2 AS FLOAT)]",
- "{k1=1.0, k2=2.0}")
+ testSqlApi("MAP['k1', CAST(1 AS DOUBLE), 'k2', CAST(2 AS FLOAT)]", "{k1=1.0, k2=2.0}")
}
@Test
def testMapField(): Unit = {
- testSqlApi(
+ testAllApis(
+ map('f4, 'f5),
+ "map(f4, f5)",
"MAP[f4, f5]",
"{foo=12}")
- testSqlApi(
+ testAllApis(
+ map('f4, 'f1),
+ "map(f4, f1)",
"MAP[f4, f1]",
"{foo={}}")
- testSqlApi(
+ testAllApis(
+ map('f2, 'f3),
+ "map(f2, f3)",
"MAP[f2, f3]",
"{{a=12, b=13}={12=a, 13=b}}")
- testSqlApi(
+ testAllApis(
+ map('f1.at("a"), 'f5),
+ "map(f1.at('a'), f5)",
"MAP[f1['a'], f5]",
"{null=12}")
- testSqlApi(
+ testAllApis(
+ 'f1,
+ "f1",
"f1",
"{}")
- testSqlApi(
+ testAllApis(
+ 'f2,
+ "f2",
"f2",
"{a=12, b=13}")
- testSqlApi(
+ testAllApis(
+ 'f2.at("a"),
+ "f2.at('a')",
"f2['a']",
"12")
- testSqlApi(
+ testAllApis(
+ 'f3.at(12),
+ "f3.at(12)",
"f3[12]",
"a")
- testSqlApi(
+ testAllApis(
+ map('f4, 'f3).at("foo").at(13),
+ "map(f4, f3).at('foo').at(13)",
"MAP[f4, f3]['foo'][13]",
"b")
}
@@ -123,72 +163,115 @@ class MapTypeTest extends MapTypeTestBase {
def testMapOperations(): Unit = {
// comparison
- testSqlApi(
+ testAllApis(
+ 'f1 === 'f2,
+ "f1 === f2",
"f1 = f2",
"false")
- testSqlApi(
+ testAllApis(
+ 'f3 === 'f7,
+ "f3 === f7",
"f3 = f7",
"true")
- testSqlApi(
+ testAllApis(
+ 'f5 === 'f2.at("a"),
+ "f5 === f2.at('a')",
"f5 = f2['a']",
"true")
- testSqlApi(
+ testAllApis(
+ 'f8 === 'f9,
+ "f8 === f9",
"f8 = f9",
"true")
- testSqlApi(
+ testAllApis(
+ 'f10 === 'f11,
+ "f10 === f11",
"f10 = f11",
"true")
- testSqlApi(
+ testAllApis(
+ 'f8 !== 'f9,
+ "f8 !== f9",
"f8 <> f9",
"false")
- testSqlApi(
+ testAllApis(
+ 'f10 !== 'f11,
+ "f10 !== f11",
"f10 <> f11",
"false")
- testSqlApi(
+ testAllApis(
+ 'f0.at("map is null"),
+ "f0.at('map is null')",
"f0['map is null']",
"null")
- testSqlApi(
+ testAllApis(
+ 'f1.at("map is empty"),
+ "f1.at('map is empty')",
"f1['map is empty']",
"null")
- testSqlApi(
+ testAllApis(
+ 'f2.at("b"),
+ "f2.at('b')",
"f2['b']",
"13")
- testSqlApi(
+ testAllApis(
+ 'f3.at(1),
+ "f3.at(1)",
"f3[1]",
"null")
- testSqlApi(
+ testAllApis(
+ 'f3.at(12),
+ "f3.at(12)",
"f3[12]",
"a")
- testSqlApi(
+ testAllApis(
+ 'f3.cardinality(),
+ "f3.cardinality()",
"CARDINALITY(f3)",
"2")
- testSqlApi(
+ testAllApis(
+ 'f2.at("a").isNotNull,
+ "f2.at('a').isNotNull",
"f2['a'] IS NOT NULL",
"true")
- testSqlApi(
+ testAllApis(
+ 'f2.at("a").isNull,
+ "f2.at('a').isNull",
"f2['a'] IS NULL",
"false")
- testSqlApi(
+ testAllApis(
+ 'f2.at("c").isNotNull,
+ "f2.at('c').isNotNull",
"f2['c'] IS NOT NULL",
"false")
- testSqlApi(
+ testAllApis(
+ 'f2.at("c").isNull,
+ "f2.at('c').isNull",
"f2['c'] IS NULL",
"true")
}
+
+ @Test
+ def testMapTypeCasting(): Unit = {
+ testTableApi(
+ 'f2.cast(DataTypes.MAP(DataTypes.STRING, DataTypes.INT)),
+ "f2.cast(MAP(STRING, INT))",
+ "{a=12, b=13}"
+ )
+ }
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
deleted file mode 100644
index 01dcf63..0000000
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
+++ /dev/null
@@ -1,694 +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.table.planner.expressions.utils.ScalarTypesTestBase
-
-import org.junit.Test
-
-class MathFunctionsTest extends ScalarTypesTestBase {
-
- // ----------------------------------------------------------------------------------------------
- // Math functions
- // ----------------------------------------------------------------------------------------------
-
- @Test
- def testMod(): Unit = {
- testSqlApi(
- "MOD(f4, f7)",
- "2")
-
- testSqlApi(
- "MOD(f4, 3)",
- "2")
-
- testSqlApi(
- "MOD(44, 3)",
- "2")
- }
-
- @Test
- def testExp(): Unit = {
- testSqlApi(
- "EXP(f2)",
- math.exp(42.toByte).toString)
-
- testSqlApi(
- "EXP(f3)",
- math.exp(43.toShort).toString)
-
- testSqlApi(
- "EXP(f4)",
- math.exp(44.toLong).toString)
-
- testSqlApi(
- "EXP(f5)",
- math.exp(4.5.toFloat).toString)
-
- testSqlApi(
- "EXP(f6)",
- math.exp(4.6).toString)
-
- testSqlApi(
- "EXP(f7)",
- math.exp(3).toString)
-
- testSqlApi(
- "EXP(3)",
- math.exp(3).toString)
- }
-
- @Test
- def testLog10(): Unit = {
- testSqlApi(
- "LOG10(f2)",
- math.log10(42.toByte).toString)
-
- testSqlApi(
- "LOG10(f3)",
- math.log10(43.toShort).toString)
-
- testSqlApi(
- "LOG10(f4)",
- math.log10(44.toLong).toString)
-
- testSqlApi(
- "LOG10(f5)",
- math.log10(4.5.toFloat).toString)
-
- testSqlApi(
- "LOG10(f6)",
- math.log10(4.6).toString)
-
- testSqlApi(
- "LOG10(f32)",
- math.log10(-1).toString)
-
- testSqlApi(
- "LOG10(f27)",
- math.log10(0).toString)
- }
-
- @Test
- def testPower(): Unit = {
- // f7: int , f4: long, f6: double
- testSqlApi(
- "POWER(f2, f7)",
- math.pow(42.toByte, 3).toString)
-
- testSqlApi(
- "POWER(f3, f6)",
- math.pow(43.toShort, 4.6D).toString)
-
- testSqlApi(
- "POWER(f4, f5)",
- math.pow(44.toLong, 4.5.toFloat).toString)
-
- testSqlApi(
- "POWER(f4, f5)",
- math.pow(44.toLong, 4.5.toFloat).toString)
-
- // f5: float
- testSqlApi(
- "power(f5, f5)",
- math.pow(4.5F, 4.5F).toString)
-
- testSqlApi(
- "power(f5, f6)",
- math.pow(4.5F, 4.6D).toString)
-
- testSqlApi(
- "power(f5, f7)",
- math.pow(4.5F, 3).toString)
-
- testSqlApi(
- "power(f5, f4)",
- math.pow(4.5F, 44L).toString)
-
- // f22: bigDecimal
- // TODO delete casting in SQL when CALCITE-1467 is fixed
- testSqlApi(
- "power(CAST(f22 AS DOUBLE), f5)",
- math.pow(2, 4.5F).toString)
-
- testSqlApi(
- "power(CAST(f22 AS DOUBLE), f6)",
- math.pow(2, 4.6D).toString)
-
- testSqlApi(
- "power(CAST(f22 AS DOUBLE), f7)",
- math.pow(2, 3).toString)
-
- testSqlApi(
- "power(CAST(f22 AS DOUBLE), f4)",
- math.pow(2, 44L).toString)
-
- testSqlApi(
- "power(f6, f22)",
- math.pow(4.6D, 2).toString)
- }
-
- @Test
- def testSqrt(): Unit = {
- testSqlApi(
- "SQRT(f6)",
- math.sqrt(4.6D).toString)
-
- testSqlApi(
- "SQRT(f7)",
- math.sqrt(3).toString)
-
- testSqlApi(
- "SQRT(f4)",
- math.sqrt(44L).toString)
-
- testSqlApi(
- "SQRT(CAST(f22 AS DOUBLE))",
- math.sqrt(2.0).toString)
-
- testSqlApi(
- "SQRT(f5)",
- math.pow(4.5F, 0.5).toString)
-
- testSqlApi(
- "SQRT(25)",
- "5.0")
-
- testSqlApi(
- "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))", // TODO fix FLINK-4621
- math.sqrt(2.2).toString)
- }
-
- @Test
- def testLn(): Unit = {
- testSqlApi(
- "LN(f2)",
- math.log(42.toByte).toString)
-
- testSqlApi(
- "LN(f3)",
- math.log(43.toShort).toString)
-
- testSqlApi(
- "LN(f4)",
- math.log(44.toLong).toString)
-
- testSqlApi(
- "LN(f5)",
- math.log(4.5.toFloat).toString)
-
- testSqlApi(
- "LN(f6)",
- math.log(4.6).toString)
-
- testSqlApi(
- "LN(f32)",
- math.log(-1).toString)
-
- testSqlApi(
- "LN(f27)",
- math.log(0).toString)
- }
-
- @Test
- def testAbs(): Unit = {
- testSqlApi(
- "ABS(f2)",
- "42")
-
- testSqlApi(
- "ABS(f3)",
- "43")
-
- testSqlApi(
- "ABS(f4)",
- "44")
-
- testSqlApi(
- "ABS(f5)",
- "4.5")
-
- testSqlApi(
- "ABS(f6)",
- "4.6")
-
- testSqlApi(
- "ABS(f9)",
- "42")
-
- testSqlApi(
- "ABS(f10)",
- "43")
-
- testSqlApi(
- "ABS(f11)",
- "44")
-
- testSqlApi(
- "ABS(f12)",
- "4.5")
-
- testSqlApi(
- "ABS(f13)",
- "4.6")
-
- testSqlApi(
- "ABS(f15)",
- "1231.1231231321321321111")
- }
-
- @Test
- def testArithmeticFloorCeil(): Unit = {
- testSqlApi(
- "FLOOR(f5)",
- "4.0")
-
- testSqlApi(
- "CEIL(f5)",
- "5.0")
-
- testSqlApi(
- "FLOOR(f3)",
- "43")
-
- testSqlApi(
- "CEIL(f3)",
- "43")
-
- testSqlApi(
- "FLOOR(f15)",
- "-1232")
-
- testSqlApi(
- "CEIL(f15)",
- "-1231")
- }
-
- @Test
- def testSin(): Unit = {
- testSqlApi(
- "SIN(f2)",
- math.sin(42.toByte).toString)
-
- testSqlApi(
- "SIN(f3)",
- math.sin(43.toShort).toString)
-
- testSqlApi(
- "SIN(f4)",
- math.sin(44.toLong).toString)
-
- testSqlApi(
- "SIN(f5)",
- math.sin(4.5.toFloat).toString)
-
- testSqlApi(
- "SIN(f6)",
- math.sin(4.6).toString)
-
- testSqlApi(
- "SIN(f15)",
- math.sin(-1231.1231231321321321111).toString)
- }
-
- @Test
- def testCos(): Unit = {
- testSqlApi(
- "COS(f2)",
- math.cos(42.toByte).toString)
-
- testSqlApi(
- "COS(f3)",
- math.cos(43.toShort).toString)
-
- testSqlApi(
- "COS(f4)",
- math.cos(44.toLong).toString)
-
- testSqlApi(
- "COS(f5)",
- math.cos(4.5.toFloat).toString)
-
- testSqlApi(
- "COS(f6)",
- math.cos(4.6).toString)
-
- testSqlApi(
- "COS(f15)",
- math.cos(-1231.1231231321321321111).toString)
- }
-
- @Test
- def testTan(): Unit = {
- testSqlApi(
- "TAN(f2)",
- math.tan(42.toByte).toString)
-
- testSqlApi(
- "TAN(f3)",
- math.tan(43.toShort).toString)
-
- testSqlApi(
- "TAN(f4)",
- math.tan(44.toLong).toString)
-
- testSqlApi(
- "TAN(f5)",
- math.tan(4.5.toFloat).toString)
-
- testSqlApi(
- "TAN(f6)",
- math.tan(4.6).toString)
-
- testSqlApi(
- "TAN(f15)",
- math.tan(-1231.1231231321321321111).toString)
- }
-
- @Test
- def testCot(): Unit = {
- testSqlApi(
- "COT(f2)",
- (1.0d / math.tan(42.toByte)).toString)
-
- testSqlApi(
- "COT(f3)",
- (1.0d / math.tan(43.toShort)).toString)
-
- testSqlApi(
- "COT(f4)",
- (1.0d / math.tan(44.toLong)).toString)
-
- testSqlApi(
- "COT(f5)",
- (1.0d / math.tan(4.5.toFloat)).toString)
-
- testSqlApi(
- "COT(f6)",
- (1.0d / math.tan(4.6)).toString)
-
- testSqlApi(
- "COT(f15)",
- (1.0d / math.tan(-1231.1231231321321321111)).toString)
- }
-
- @Test
- def testAsin(): Unit = {
- testSqlApi(
- "ASIN(f25)",
- math.asin(0.42.toByte).toString)
-
- testSqlApi(
- "ASIN(f26)",
- math.asin(0.toShort).toString)
-
- testSqlApi(
- "ASIN(f27)",
- math.asin(0.toLong).toString)
-
- testSqlApi(
- "ASIN(f28)",
- math.asin(0.45.toFloat).toString)
-
- testSqlApi(
- "ASIN(f29)",
- math.asin(0.46).toString)
-
- testSqlApi(
- "ASIN(f30)",
- math.asin(1).toString)
-
- testSqlApi(
- "ASIN(f31)",
- math.asin(-0.1231231321321321111).toString)
- }
-
- @Test
- def testAcos(): Unit = {
- testSqlApi(
- "ACOS(f25)",
- math.acos(0.42.toByte).toString)
-
- testSqlApi(
- "ACOS(f26)",
- math.acos(0.toShort).toString)
-
- testSqlApi(
- "ACOS(f27)",
- math.acos(0.toLong).toString)
-
- testSqlApi(
- "ACOS(f28)",
- math.acos(0.45.toFloat).toString)
-
- testSqlApi(
- "ACOS(f29)",
- math.acos(0.46).toString)
-
- testSqlApi(
- "ACOS(f30)",
- math.acos(1).toString)
-
- testSqlApi(
- "ACOS(f31)",
- math.acos(-0.1231231321321321111).toString)
- }
-
- @Test
- def testAtan(): Unit = {
- testSqlApi(
- "ATAN(f25)",
- math.atan(0.42.toByte).toString)
-
- testSqlApi(
- "ATAN(f26)",
- math.atan(0.toShort).toString)
-
- testSqlApi(
- "ATAN(f27)",
- math.atan(0.toLong).toString)
-
- testSqlApi(
- "ATAN(f28)",
- math.atan(0.45.toFloat).toString)
-
- testSqlApi(
- "ATAN(f29)",
- math.atan(0.46).toString)
-
- testSqlApi(
- "ATAN(f30)",
- math.atan(1).toString)
-
- testSqlApi(
- "ATAN(f31)",
- math.atan(-0.1231231321321321111).toString)
- }
-
- @Test
- def testDegrees(): Unit = {
- testSqlApi(
- "DEGREES(f2)",
- math.toDegrees(42.toByte).toString)
-
- testSqlApi(
- "DEGREES(f3)",
- math.toDegrees(43.toShort).toString)
-
- testSqlApi(
- "DEGREES(f4)",
- math.toDegrees(44.toLong).toString)
-
- testSqlApi(
- "DEGREES(f5)",
- math.toDegrees(4.5.toFloat).toString)
-
- testSqlApi(
- "DEGREES(f6)",
- math.toDegrees(4.6).toString)
-
- testSqlApi(
- "DEGREES(f15)",
- math.toDegrees(-1231.1231231321321321111).toString)
- }
-
- @Test
- def testRadians(): Unit = {
- testSqlApi(
- "RADIANS(f2)",
- math.toRadians(42.toByte).toString)
-
- testSqlApi(
- "RADIANS(f3)",
- math.toRadians(43.toShort).toString)
-
- testSqlApi(
- "RADIANS(f4)",
- math.toRadians(44.toLong).toString)
-
- testSqlApi(
- "RADIANS(f5)",
- math.toRadians(4.5.toFloat).toString)
-
- testSqlApi(
- "RADIANS(f6)",
- math.toRadians(4.6).toString)
-
- testSqlApi(
- "RADIANS(f15)",
- math.toRadians(-1231.1231231321321321111).toString)
- }
-
- @Test
- def testSign(): Unit = {
- testSqlApi(
- "SIGN(f4)",
- 1.toString)
-
- testSqlApi(
- "SIGN(f6)",
- 1.0.toString)
-
- testSqlApi(
- "SIGN(f15)",
- "-1.0000000000000000000") // calcite: SIGN(Decimal(p,s)) => Decimal(p,s)
- }
-
- @Test
- def testRound(): Unit = {
- testSqlApi(
- "ROUND(f29, f30)",
- 0.5.toString)
-
- testSqlApi(
- "ROUND(f31, f7)",
- "-0.123")
-
- testSqlApi(
- "ROUND(f4, f32)",
- 40.toString)
- }
-
- @Test
- def testPi(): Unit = {
- testSqlApi(
- "pi()",
- math.Pi.toString)
- }
-
- @Test
- def testRandAndRandInteger(): Unit = {
- val random1 = new java.util.Random(1)
- testSqlApi(
- "RAND(1)",
- random1.nextDouble().toString)
-
- val random2 = new java.util.Random(3)
- testSqlApi(
- "RAND(f7)",
- random2.nextDouble().toString)
-
- val random3 = new java.util.Random(1)
- testSqlApi(
- "RAND_INTEGER(1, 10)",
- random3.nextInt(10).toString)
-
- val random4 = new java.util.Random(3)
- testSqlApi(
- "RAND_INTEGER(f7, CAST(f4 AS INT))",
- random4.nextInt(44).toString)
- }
-
- @Test
- def testE(): Unit = {
- testSqlApi(
- "E()",
- math.E.toString)
-
- testSqlApi(
- "e()",
- math.E.toString)
- }
-
- @Test
- def testLog(): Unit = {
- testSqlApi(
- "LOG(f6)",
- "1.5260563034950492"
- )
-
- testSqlApi(
- "LOG(f6-f6 + 10, f6-f6+100)",
- "2.0"
- )
-
- testSqlApi(
- "LOG(f6+20)",
- "3.202746442938317"
- )
-
- testSqlApi(
- "LOG(10)",
- "2.302585092994046"
- )
-
- testSqlApi(
- "LOG(10, 100)",
- "2.0"
- )
-
- testSqlApi(
- "log(f32, f32)",
- (math.log(-1)/math.log(-1)).toString)
-
- testSqlApi(
- "log(f27, f32)",
- (math.log(0)/math.log(0)).toString)
- }
-
- @Test
- def testLog2(): Unit = {
- testSqlApi(
- "log2(f2)",
- (math.log(42.toByte)/math.log(2.toByte)).toString)
-
- testSqlApi(
- "log2(f3)",
- (math.log(43.toShort)/math.log(2.toShort)).toString)
-
- testSqlApi(
- "log2(f4)",
- (math.log(44.toLong)/math.log(2.toLong)).toString)
-
- testSqlApi(
- "log2(f5)",
- (math.log(4.5.toFloat)/math.log(2.toFloat)).toString)
-
- testSqlApi(
- "log2(f6)",
- (math.log(4.6)/math.log(2)).toString)
-
- testSqlApi(
- "log2(f32)",
- (math.log(-1)/math.log(2)).toString)
-
- testSqlApi(
- "log2(f27)",
- (math.log(0)/math.log(2)).toString)
- }
-}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
index 98c64ad..2565383 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.expressions
import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
import org.apache.flink.types.Row
@@ -27,47 +28,64 @@ import org.junit.{Ignore, Test}
/**
* Tests that can only be checked manually as they are non-deterministic.
*/
-@Ignore
class NonDeterministicTests extends ExpressionTestBase {
+ @Ignore
@Test
def testCurrentDate(): Unit = {
- testSqlApi(
+ testAllApis(
+ currentDate(),
+ "currentDate()",
"CURRENT_DATE",
"PLEASE CHECK MANUALLY")
}
+ @Ignore
@Test
def testCurrentTime(): Unit = {
- testSqlApi(
+ testAllApis(
+ currentTime(),
+ "currentTime()",
"CURRENT_TIME",
"PLEASE CHECK MANUALLY")
}
+ @Ignore
@Test
def testCurrentTimestamp(): Unit = {
- testSqlApi(
+ testAllApis(
+ currentTimestamp(),
+ "currentTimestamp()",
"CURRENT_TIMESTAMP",
"PLEASE CHECK MANUALLY")
}
+ @Ignore
@Test
def testLocalTimestamp(): Unit = {
- testSqlApi(
+ testAllApis(
+ localTimestamp(),
+ "localTimestamp()",
"LOCALTIMESTAMP",
"PLEASE CHECK MANUALLY")
}
+ @Ignore
@Test
def testLocalTime(): Unit = {
- testSqlApi(
+ testAllApis(
+ localTime(),
+ "localTime()",
"LOCALTIME",
"PLEASE CHECK MANUALLY")
}
+ @Ignore
@Test
def testUUID(): Unit = {
- testSqlApi(
+ testAllApis(
+ uuid(),
+ "uuid()",
"UUID()",
"PLEASE CHECK MANUALLY")
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
index 2c6f924..ff7a2a2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
@@ -18,7 +18,10 @@
package org.apache.flink.table.planner.expressions
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.RowTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
import org.junit.Test
@@ -28,17 +31,36 @@ class RowTypeTest extends RowTypeTestBase {
def testRowLiteral(): Unit = {
// primitive literal
- testSqlApi(
+ testAllApis(
+ row(1, "foo", true),
+ "row(1, 'foo', true)",
"ROW(1, 'foo', true)",
"(1,foo,true)")
// special literal
- testSqlApi(
+ testAllApis(
+ row(
+ localDate("1985-04-11"),
+ gLocalTime("14:15:16"),
+ localDateTime("1985-04-11 14:15:16"),
+ BigDecimal("0.1").bigDecimal,
+ array(1, 2, 3),
+ map("foo", "bar"),
+ row(1, true)),
+ "row('1985-04-11'.toDate, '14:15:16'.toTime, '1985-04-11 14:15:16'.toTimestamp, " +
+ "0.1p, Array(1, 2, 3), Map('foo', 'bar'), row(1, true))",
"ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
- "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
+ "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
"(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))")
testSqlApi(
+ "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
+ "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
+ "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))")
+
+ testAllApis(
+ row(1 + 1, 2 * 3, nullOf(DataTypes.STRING())),
+ "row(1 + 1, 2 * 3, Null(STRING))",
"ROW(1 + 1, 2 * 3, NULLIF(1, 1))",
"(2,6,null)"
)
@@ -48,27 +70,37 @@ class RowTypeTest extends RowTypeTestBase {
@Test
def testRowField(): Unit = {
- testSqlApi(
+ testAllApis(
+ row('f0, 'f1),
+ "row(f0, f1)",
"(f0, f1)",
"(null,1)"
)
- testSqlApi(
+ testAllApis(
+ 'f2,
+ "f2",
"f2",
"(2,foo,true)"
)
- testSqlApi(
+ testAllApis(
+ row('f2, 'f5),
+ "row(f2, f5)",
"(f2, f5)",
"((2,foo,true),(foo,null))"
)
- testSqlApi(
+ testAllApis(
+ 'f4,
+ "f4",
"f4",
"(1984-03-12,0.00000000,[1, 2, 3])"
)
- testSqlApi(
+ testAllApis(
+ row('f1, "foo", true),
+ "row(f1, 'foo', true)",
"(f1, 'foo',true)",
"(1,foo,true)"
)
@@ -76,14 +108,25 @@ class RowTypeTest extends RowTypeTestBase {
@Test
def testRowOperations(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f5.get("f0"),
+ "f5.get('f0')",
"f5.f0",
"foo"
)
- testSqlApi(
+ testAllApis(
+ 'f3.get("f1").get("f2"),
+ "f3.get('f1').get('f2')",
"f3.f1.f2",
"true"
)
+
+ // SQL API for row value constructor follow by field access is not supported
+ testTableApi(
+ row('f1, 'f6, 'f2).get("f1").get("f1"),
+ "row(f1, f6, f2).get('f1').get('f1')",
+ "null"
+ )
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
index 5c3b0f9..244812a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
@@ -18,7 +18,9 @@
package org.apache.flink.table.planner.expressions
-import org.apache.flink.table.expressions.TimePointUnit
+import org.apache.flink.table.api.scala.{currentDate, currentTime, currentTimestamp, localTime, localTimestamp, nullOf, temporalOverlaps, _}
+import org.apache.flink.table.api.{DataTypes, Types}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser, TimeIntervalUnit, TimePointUnit}
import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase
import org.junit.Test
@@ -110,8 +112,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
)
cases.foreach(x => {
- // TODO: ignore Table API currently
- testSqlApi(
+ testAllApis(
+ ExpressionParser.parseExpression(x._1),
+ x._1,
x._2,
x._3
)
@@ -121,17 +124,23 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testPosition(): Unit = {
- testSqlApi(
+ testAllApis(
+ "test".position("xxxtest"),
+ "'test'.position('xxxtest')",
"POSITION('test' IN 'xxxtest')",
"4")
- testSqlApi(
+ testAllApis(
+ "testx".position("xxxtest"),
+ "'testx'.position('xxxtest')",
"POSITION('testx' IN 'xxxtest')",
"0")
testSqlApi(
"POSITION('aa' IN 'aaads')",
"1")
+
+ testSqlApi("position('aa' in 'aaads')", "1")
}
@Test
@@ -289,19 +298,27 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testSubstring(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.substring(2),
+ "f0.substring(2)",
"SUBSTRING(f0, 2)",
"his is a test String.")
- testSqlApi(
+ testAllApis(
+ 'f0.substring(2, 5),
+ "f0.substring(2, 5)",
"SUBSTRING(f0, 2, 5)",
"his i")
- testSqlApi(
+ testAllApis(
+ 'f0.substring(1, 'f7),
+ "f0.substring(1, f7)",
"SUBSTRING(f0, 1, f7)",
"Thi")
- testSqlApi(
+ testAllApis(
+ 'f0.substring(1.cast(DataTypes.TINYINT), 'f7),
+ "f0.substring(1.cast(BYTE), f7)",
"SUBSTRING(f0, CAST(1 AS TINYINT), f7)",
"Thi")
@@ -327,20 +344,61 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
}
@Test
+ def testReplace(): Unit = {
+ testAllApis(
+ 'f0.replace(" ", "_"),
+ "f0.replace(' ', '_')",
+ "REPLACE(f0, ' ', '_')",
+ "This_is_a_test_String.")
+
+ testAllApis(
+ 'f0.replace("i", ""),
+ "f0.replace('i', '')",
+ "REPLACE(f0, 'i', '')",
+ "Ths s a test Strng.")
+
+ testAllApis(
+ 'f33.replace("i", ""),
+ "f33.replace('i', '')",
+ "REPLACE(f33, 'i', '')",
+ "null")
+
+ testAllApis(
+ 'f0.replace(nullOf(DataTypes.STRING), ""),
+ "f0.replace(Null(STRING), '')",
+ "REPLACE(f0, NULLIF('', ''), '')",
+ "null")
+
+ testAllApis(
+ 'f0.replace(" ", nullOf(DataTypes.STRING)),
+ "f0.replace(' ', Null(STRING))",
+ "REPLACE(f0, ' ', NULLIF('', ''))",
+ "null")
+ }
+
+ @Test
def testTrim(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f8.trim(),
+ "f8.trim()",
"TRIM(f8)",
"This is a test String.")
- testSqlApi(
+ testAllApis(
+ 'f8.trim(removeLeading = true, removeTrailing = true, " "),
+ "trim(f8)",
"TRIM(f8)",
"This is a test String.")
- testSqlApi(
+ testAllApis(
+ 'f8.trim(removeLeading = false, removeTrailing = true, " "),
+ "f8.trim(TRAILING, ' ')",
"TRIM(TRAILING FROM f8)",
" This is a test String.")
- testSqlApi(
+ testAllApis(
+ 'f0.trim(removeLeading = true, removeTrailing = true, "."),
+ "trim(BOTH, '.', f0)",
"TRIM(BOTH '.' FROM f0)",
"This is a test String")
@@ -373,7 +431,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testLTrim(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f8.ltrim(),
+ "f8.ltrim()",
"LTRIM(f8)",
"This is a test String. ")
@@ -408,7 +468,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testRTrim(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f8.rtrim(),
+ "f8.rtrim()",
"rtrim(f8)",
" This is a test String.")
@@ -443,11 +505,15 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testCharLength(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.charLength(),
+ "f0.charLength()",
"CHAR_LENGTH(f0)",
"22")
- testSqlApi(
+ testAllApis(
+ 'f0.charLength(),
+ "charLength(f0)",
"CHARACTER_LENGTH(f0)",
"22")
}
@@ -469,21 +535,27 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testUpperCase(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.upperCase(),
+ "f0.upperCase()",
"UPPER(f0)",
"THIS IS A TEST STRING.")
}
@Test
def testLowerCase(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.lowerCase(),
+ "f0.lowerCase()",
"LOWER(f0)",
"this is a test string.")
}
@Test
def testInitCap(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.initCap(),
+ "f0.initCap()",
"INITCAP(f0)",
"This Is A Test String.")
@@ -494,18 +566,24 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testConcat(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0 + 'f0,
+ "f0 + f0",
"f0||f0",
"This is a test String.This is a test String.")
}
@Test
def testLike(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.like("Th_s%"),
+ "f0.like('Th_s%')",
"f0 LIKE 'Th_s%'",
"true")
- testSqlApi(
+ testAllApis(
+ 'f0.like("%is a%"),
+ "f0.like('%is a%')",
"f0 LIKE '%is a%'",
"true")
@@ -529,11 +607,15 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testNotLike(): Unit = {
- testSqlApi(
+ testAllApis(
+ !'f0.like("Th_s%"),
+ "!f0.like('Th_s%')",
"f0 NOT LIKE 'Th_s%'",
"false")
- testSqlApi(
+ testAllApis(
+ !'f0.like("%is a%"),
+ "!f0.like('%is a%')",
"f0 NOT LIKE '%is a%'",
"false")
}
@@ -578,22 +660,30 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testSimilar(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.similar("_*"),
+ "f0.similar('_*')",
"f0 SIMILAR TO '_*'",
"true")
- testSqlApi(
+ testAllApis(
+ 'f0.similar("This (is)? a (test)+ Strin_*"),
+ "f0.similar('This (is)? a (test)+ Strin_*')",
"f0 SIMILAR TO 'This (is)? a (test)+ Strin_*'",
"true")
}
@Test
def testNotSimilar(): Unit = {
- testSqlApi(
+ testAllApis(
+ !'f0.similar("_*"),
+ "!f0.similar('_*')",
"f0 NOT SIMILAR TO '_*'",
"false")
- testSqlApi(
+ testAllApis(
+ !'f0.similar("This (is)? a (test)+ Strin_*"),
+ "!f0.similar('This (is)? a (test)+ Strin_*')",
"f0 NOT SIMILAR TO 'This (is)? a (test)+ Strin_*'",
"false")
}
@@ -638,13 +728,19 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testMultiConcat(): Unit = {
- testSqlApi(
+ testAllApis(
+ concat("xx", 'f33),
+ "concat('xx', f33)",
"CONCAT('xx', f33)",
"xx")
- testSqlApi(
+ testAllApis(
+ concat("AA", "BB", "CC", "---"),
+ "concat('AA','BB','CC','---')",
"CONCAT('AA','BB','CC','---')",
"AABBCC---")
- testSqlApi(
+ testAllApis(
+ concat("x~x", "b~b", "c~~~~c", "---"),
+ "concat('x~x','b~b','c~~~~c','---')",
"CONCAT('x~x','b~b','c~~~~c','---')",
"x~xb~bc~~~~c---")
@@ -655,124 +751,156 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testConcatWs(): Unit = {
- testSqlApi(
+ testAllApis(
+ concat_ws('f33, "AA"),
+ "concat_ws(f33, 'AA')",
"CONCAT_WS(f33, 'AA')",
"AA")
- testSqlApi(
+ testAllApis(
+ concat_ws("~~~~", "AA"),
+ "concat_ws('~~~~','AA')",
"concat_ws('~~~~','AA')",
"AA")
- testSqlApi(
+ testAllApis(
+ concat_ws("~", "AA", "BB"),
+ "concat_ws('~','AA','BB')",
"concat_ws('~','AA','BB')",
"AA~BB")
- testSqlApi(
+ testAllApis(
+ concat_ws("~", 'f33, "AA", "BB", "", 'f33, "CC"),
+ "concat_ws('~',f33, 'AA','BB','',f33, 'CC')",
"concat_ws('~',f33, 'AA','BB','',f33, 'CC')",
"AA~BB~~CC")
- testSqlApi(
+ testAllApis(
+ concat_ws("~~~~", "Flink", 'f33, "xx", 'f33, 'f33),
+ "concat_ws('~~~~','Flink', f33, 'xx', f33, f33)",
"CONCAT_WS('~~~~','Flink', f33, 'xx', f33, f33)",
"Flink~~~~xx")
testSqlApi("concat_ws('||', f35, f36, f33)", "a||b")
}
+ @Test
def testRegexpReplace(): Unit = {
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace("oo|ar", "abc"),
+ "'foobar'.regexpReplace('oo|ar', 'abc')",
"regexp_replace('foobar', 'oo|ar', 'abc')",
"fabcbabc")
- testSqlApi(
+ testAllApis(
+ "foofar".regexpReplace("^f", ""),
+ "'foofar'.regexpReplace('^f', '')",
"regexp_replace('foofar', '^f', '')",
"oofar")
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace("^f*.*r$", ""),
+ "'foobar'.regexpReplace('^f*.*r$', '')",
"regexp_replace('foobar', '^f*.*r$', '')",
"")
- testSqlApi(
+ testAllApis(
+ "foo1bar2".regexpReplace("\\d", ""),
+ "'foo1bar2'.regexpReplace('\\d', '')",
"regexp_replace('foobar', '\\d', '')",
"foobar")
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace("\\w", ""),
+ "'foobar'.regexpReplace('\\w', '')",
"regexp_replace('foobar', '\\w', '')",
"")
- testSqlApi(
+ testAllApis(
+ "fooobar".regexpReplace("oo", "$"),
+ "'fooobar'.regexpReplace('oo', '$')",
"regexp_replace('fooobar', 'oo', '$')",
"f$obar")
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace("oo", "\\"),
+ "'foobar'.regexpReplace('oo', '\\')",
"regexp_replace('foobar', 'oo', '\\')",
"f\\bar")
- testSqlApi(
+ testAllApis(
+ 'f33.regexpReplace("oo|ar", ""),
+ "f33.regexpReplace('oo|ar', '')",
"REGEXP_REPLACE(f33, 'oo|ar', '')",
"null")
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace('f33, ""),
+ "'foobar'.regexpReplace(f33, '')",
"REGEXP_REPLACE('foobar', f33, '')",
"null")
- testSqlApi(
+ testAllApis(
+ "foobar".regexpReplace("oo|ar", 'f33),
+ "'foobar'.regexpReplace('oo|ar', f33)",
"REGEXP_REPLACE('foobar', 'oo|ar', f33)",
"null")
// This test was added for the null literal problem in string expression parsing (FLINK-10463).
- testSqlApi(
+ testAllApis(
+ nullOf(Types.STRING).regexpReplace("oo|ar", 'f33),
+ "nullOf(STRING).regexpReplace('oo|ar', f33)",
"REGEXP_REPLACE(CAST(NULL AS VARCHAR), 'oo|ar', f33)",
"null")
-
- testSqlApi("regexp_replace('100-200', '(\\d+)', 'num')", "num-num")
- testSqlApi("regexp_replace('100-200', '(\\d+)-(\\d+)', '400')", "400")
- testSqlApi("regexp_replace('100-200', '(\\d+)', '400')", "400-400")
- testSqlApi("regexp_replace('100-200', '', '400')", "100-200")
- testSqlApi("regexp_replace(f40, '(\\d+)', '400')", "null")
- testSqlApi("regexp_replace(CAST(null as VARCHAR), '(\\d+)', 'num')", "null")
- testSqlApi("regexp_replace('100-200', CAST(null as VARCHAR), '400')", "null")
- testSqlApi("regexp_replace('100-200', '(\\d+)', CAST(null as VARCHAR))", "null")
}
@Test
def testRegexpExtract(): Unit = {
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo(.*?)(bar)", 2),
+ "'foothebar'.regexpExtract('foo(.*?)(bar)', 2)",
"REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 2)",
"bar")
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo(.*?)(bar)", 0),
+ "'foothebar'.regexpExtract('foo(.*?)(bar)', 0)",
"REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 0)",
"foothebar")
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo(.*?)(bar)", 1),
+ "'foothebar'.regexpExtract('foo(.*?)(bar)', 1)",
"REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 1)",
"the")
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo([\\w]+)", 1),
+ "'foothebar'.regexpExtract('foo([\\w]+)', 1)",
"REGEXP_EXTRACT('foothebar', 'foo([\\w]+)', 1)",
"thebar")
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo([\\d]+)", 1),
+ "'foothebar'.regexpExtract('foo([\\d]+)', 1)",
"REGEXP_EXTRACT('foothebar', 'foo([\\d]+)', 1)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.regexpExtract("foo(.*?)(bar)", 2),
+ "f33.regexpExtract('foo(.*?)(bar)', 2)",
"REGEXP_EXTRACT(f33, 'foo(.*?)(bar)', 2)",
"null")
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract('f33, 2),
+ "'foothebar'.regexpExtract(f33, 2)",
"REGEXP_EXTRACT('foothebar', f33, 2)",
"null")
//test for optional group index
- testSqlApi(
+ testAllApis(
+ "foothebar".regexpExtract("foo(.*?)(bar)"),
+ "'foothebar'.regexpExtract('foo(.*?)(bar)')",
"REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)')",
"foothebar")
-
- testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', 1)", "100")
- testSqlApi("regexp_extract('100-200', '', 1)", "null")
- testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', -1)", "null")
- testSqlApi("regexp_extract(f40, '(\\d+)-(\\d+)', 1)", "null")
- testSqlApi("regexp_extract(CAST(null as VARCHAR), '(\\d+)-(\\d+)', 1)", "null")
- testSqlApi("regexp_extract('100-200', CAST(null as VARCHAR), 1)", "null")
- testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', CAST(null as BIGINT))", "null")
}
@Test
@@ -825,35 +953,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
}
@Test
- def testUUID(): Unit = {
- testSqlApi(
- "CHARACTER_LENGTH(UUID())",
- "36")
-
- testSqlApi(
- "SUBSTRING(UUID(), 9, 1)",
- "-")
-
- testSqlApi(
- "SUBSTRING(UUID(), 14, 1)",
- "-")
-
- testSqlApi(
- "SUBSTRING(UUID(), 19, 1)",
- "-")
-
- testSqlApi(
- "SUBSTRING(UUID(), 24, 1)",
- "-")
-
- // test uuid with bytes
- testSqlApi(
- "UUID(f53)",
- "5eb63bbb-e01e-3ed0-93cb-22bb8f5acdc3"
- )
- }
-
- @Test
def testSubString(): Unit = {
Array("substring", "substr").foreach {
substr =>
@@ -894,7 +993,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
testSqlApi("lpad('Hello', -1, 'x') IS NOT NULL", "false")
testSqlApi("lpad('ab', 5, '')", "null")
- testSqlApi(
+ testAllApis(
+ "äää".lpad(13, "12345"),
+ "'äää'.lpad(13, '12345')",
"lpad('äää',13,'12345')",
"1234512345äää")
}
@@ -918,7 +1019,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
testSqlApi("rpad('\u0061\u0062',1,'??')", "a") // the unicode of ab is \u0061\u0062
testSqlApi("rpad('üö',1,'??')", "ü")
testSqlApi("rpad('abcd', 5, '')", "null")
- testSqlApi(
+ testAllApis(
+ "äää".rpad(13, "12345"),
+ "'äää'.rpad(13, '12345')",
"rpad('äää',13,'12345')",
"äää1234512345")
}
@@ -993,13 +1096,41 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testRepeat(): Unit = {
- testSqlApi("repeat(f35, 2)", "aa")
- testSqlApi("repeat(f35, 0)", "")
- testSqlApi("repeat(f40, 2)", "null")
- testSqlApi("repeat('hi', 2)", "hihi")
- testSqlApi("repeat('hi', 0)", "")
- testSqlApi("repeat('hi', CAST(null as INT))", "null")
- testSqlApi("repeat(CAST(null as VARCHAR), 2)", "null")
+ testAllApis(
+ 'f0.repeat(1),
+ "f0.repeat(1)",
+ "REPEAT(f0, 1)",
+ "This is a test String.")
+
+ testAllApis(
+ 'f0.repeat(2),
+ "f0.repeat(2)",
+ "REPEAT(f0, 2)",
+ "This is a test String.This is a test String.")
+
+ testAllApis(
+ 'f0.repeat(0),
+ "f0.repeat(0)",
+ "REPEAT(f0, 0)",
+ "")
+
+ testAllApis(
+ 'f0.repeat(-1),
+ "f0.repeat(-1)",
+ "REPEAT(f0, -1)",
+ "")
+
+ testAllApis(
+ 'f33.repeat(2),
+ "f33.repeat(2)",
+ "REPEAT(f33, 2)",
+ "null")
+
+ testAllApis(
+ "".repeat(1),
+ "''.repeat(1)",
+ "REPEAT('', 2)",
+ "")
}
@Test
@@ -1012,19 +1143,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
}
@Test
- def testReplace(): Unit = {
- testSqlApi("replace(f38, 'A', 'a')", "aQIDBa==")
- testSqlApi("replace(f38, 'Z', 'a')", "AQIDBA==")
- testSqlApi("replace(f38, CAST(null as VARCHAR), 'a')", "null")
- testSqlApi("replace(f38, 'A', CAST(null as VARCHAR))", "null")
- testSqlApi("replace(f40, 'A', 'a')", "null")
- testSqlApi("replace('Test', 'T', 't')", "test")
- testSqlApi("replace(CAST(null as VARCHAR), 'T', 't')", "null")
- testSqlApi("replace('Test', CAST(null as VARCHAR), 't')", "null")
- testSqlApi("replace('Test', 'T', CAST(null as VARCHAR))", "null")
- }
-
- @Test
def testSplitIndex(): Unit = {
testSqlApi("split_index(f38, 'I', 0)", "AQ")
testSqlApi("split_index(f38, 'I', 2)", "null")
@@ -1096,67 +1214,209 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
}
@Test
+ def testHex(): Unit = {
+ testAllApis(
+ 100.hex(),
+ "100.hex()",
+ "HEX(100)",
+ "64")
+
+ testAllApis(
+ 'f2.hex(),
+ "f2.hex()",
+ "HEX(f2)",
+ "2A")
+
+ testAllApis(
+ nullOf(DataTypes.TINYINT).hex(),
+ "hex(Null(BYTE))",
+ "HEX(CAST(NULL AS TINYINT))",
+ "null")
+
+ testAllApis(
+ 'f3.hex(),
+ "f3.hex()",
+ "HEX(f3)",
+ "2B")
+
+ testAllApis(
+ 'f4.hex(),
+ "f4.hex()",
+ "HEX(f4)",
+ "2C")
+
+ testAllApis(
+ 'f7.hex(),
+ "f7.hex()",
+ "HEX(f7)",
+ "3")
+
+ testAllApis(
+ 12.hex(),
+ "12.hex()",
+ "HEX(12)",
+ "C")
+
+ testAllApis(
+ 10.hex(),
+ "10.hex()",
+ "HEX(10)",
+ "A")
+
+ testAllApis(
+ 0.hex(),
+ "0.hex()",
+ "HEX(0)",
+ "0")
+
+ testAllApis(
+ "ö".hex(),
+ "'ö'.hex()",
+ "HEX('ö')",
+ "C3B6")
+
+ testAllApis(
+ 'f32.hex(),
+ "f32.hex()",
+ "HEX(f32)",
+ "FFFFFFFFFFFFFFFF")
+
+ testAllApis(
+ 'f0.hex(),
+ "f0.hex()",
+ "HEX(f0)",
+ "546869732069732061207465737420537472696E672E")
+
+ testAllApis(
+ 'f8.hex(),
+ "f8.hex()",
+ "HEX(f8)",
+ "20546869732069732061207465737420537472696E672E20")
+
+ testAllApis(
+ 'f23.hex(),
+ "f23.hex()",
+ "HEX(f23)",
+ "25546869732069732061207465737420537472696E672E")
+
+ testAllApis(
+ 'f24.hex(),
+ "f24.hex()",
+ "HEX(f24)",
+ "2A5F546869732069732061207465737420537472696E672E")
+ }
+
+ @Test
def testBin(): Unit = {
- testSqlApi(
+ testAllApis(
+ nullOf(DataTypes.TINYINT).bin(),
+ "bin(Null(BYTE))",
"BIN((CAST(NULL AS TINYINT)))",
"null")
- testSqlApi(
+ testAllApis(
+ 'f2.bin(),
+ "f2.bin()",
"BIN(f2)",
"101010")
- testSqlApi(
+ testAllApis(
+ 'f3.bin(),
+ "f3.bin()",
"BIN(f3)",
"101011")
- testSqlApi(
+ testAllApis(
+ 'f4.bin(),
+ "f4.bin()",
"BIN(f4)",
"101100")
- testSqlApi(
+ testAllApis(
+ 'f7.bin(),
+ "f7.bin()",
"BIN(f7)",
"11")
- testSqlApi(
+ testAllApis(
+ 12.bin(),
+ "12.bin()",
"BIN(12)",
"1100")
- testSqlApi(
+ testAllApis(
+ 10.bin(),
+ "10.bin()",
"BIN(10)",
"1010")
- testSqlApi(
+ testAllApis(
+ 0.bin(),
+ "0.bin()",
"BIN(0)",
"0")
- testSqlApi(
- "BIN(-7)",
- "1111111111111111111111111111111111111111111111111111111111111001")
-
- testSqlApi(
- "BIN(-1)",
- "1111111111111111111111111111111111111111111111111111111111111111")
-
- testSqlApi(
+ testAllApis(
+ 'f32.bin(),
+ "f32.bin()",
"BIN(f32)",
"1111111111111111111111111111111111111111111111111111111111111111")
}
+ @Test
+ def testUUID(): Unit = {
+ testAllApis(
+ uuid().charLength(),
+ "uuid().charLength",
+ "CHARACTER_LENGTH(UUID())",
+ "36")
+
+ testAllApis(
+ uuid().substring(9, 1),
+ "uuid().substring(9, 1)",
+ "SUBSTRING(UUID(), 9, 1)",
+ "-")
+
+ testAllApis(
+ uuid().substring(14, 1),
+ "uuid().substring(14, 1)",
+ "SUBSTRING(UUID(), 14, 1)",
+ "-")
+
+ testAllApis(
+ uuid().substring(19, 1),
+ "uuid().substring(19, 1)",
+ "SUBSTRING(UUID(), 19, 1)",
+ "-")
+
+ testAllApis(
+ uuid().substring(24, 1),
+ "uuid().substring(24, 1)",
+ "SUBSTRING(UUID(), 24, 1)",
+ "-")
+ }
+
// ----------------------------------------------------------------------------------------------
// Math functions
// ----------------------------------------------------------------------------------------------
@Test
def testAdd(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1514356320000L + 6000,
+ "1514356320000L + 6000",
"1514356320000 + 6000",
"1514356326000")
- testSqlApi(
+ testAllApis(
+ 'f34 + 6,
+ "f34 + 6",
"f34 + 6",
"1514356320006")
- testSqlApi(
+ testAllApis(
+ 'f34 + 'f34,
+ "f34 + f34",
"f34 + f34",
"3028712640000")
}
@@ -1164,15 +1424,21 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testSubtract(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1514356320000L - 6000,
+ "1514356320000L - 6000",
"1514356320000 - 6000",
"1514356314000")
- testSqlApi(
+ testAllApis(
+ 'f34 - 6,
+ "f34 - 6",
"f34 - 6",
"1514356319994")
- testSqlApi(
+ testAllApis(
+ 'f34 - 'f34,
+ "f34 - f34",
"f34 - f34",
"0")
}
@@ -1180,16 +1446,22 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testMultiply(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1514356320000L * 60000,
+ "1514356320000L * 60000",
"1514356320000 * 60000",
"90861379200000000")
- testSqlApi(
+ testAllApis(
+ 'f34 * 6,
+ "f34 * 6",
"f34 * 6",
"9086137920000")
- testSqlApi(
+ testAllApis(
+ 'f34 * 'f34,
+ "f34 * f34",
"f34 * f34",
"2293275063923942400000000")
@@ -1198,28 +1470,31 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testDivide(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1514356320000L / 60000.0, // the `/` is Scala operator, not Flink TableApi operator
+ "1514356320000L / 60000",
"1514356320000 / 60000",
"2.5239272E7")
- // DIV return decimal
- testSqlApi(
- "DIV(1514356320000, 60000)",
- "25239272")
-
- testSqlApi(
+ testAllApis(
+ 'f7 / 2,
+ "f7 / 2",
"f7 / 2",
"1.5")
// f34 => Decimal(19,0)
// 6 => Integer => Decimal(10,0)
// Decimal(19,0) / Decimal(10,0) => Decimal(30,11)
- testSqlApi(
+ testAllApis(
+ 'f34 / 6,
+ "f34 / 6",
"f34 / 6",
"252392720000.00000000000")
// Decimal(19,0) / Decimal(19,0) => Decimal(39,20) => Decimal(38,19)
- testSqlApi(
+ testAllApis(
+ 'f34 / 'f34,
+ "f34 / f34",
"f34 / f34",
"1.0000000000000000000")
}
@@ -1227,81 +1502,117 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testMod(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1514356320000L % 60000,
+ "1514356320000L % 60000",
"mod(1514356320000,60000)",
"0")
- testSqlApi(
+ testAllApis(
+ 'f34.mod('f34),
+ "f34.mod(f34)",
"mod(f34,f34)",
"0")
- testSqlApi(
+ testAllApis(
+ 'f34.mod(6),
+ "f34.mod(6)",
"mod(f34,6)",
"0")
- testSqlApi(
+ testAllApis(
+ 'f4.mod('f7),
+ "f4.mod(f7)",
"MOD(f4, f7)",
"2")
- testSqlApi(
+ testAllApis(
+ 'f4.mod(3),
+ "mod(f4, 3)",
"MOD(f4, 3)",
"2")
- testSqlApi(
+ testAllApis(
+ 'f4 % 3,
+ "mod(44, 3)",
"MOD(44, 3)",
"2")
}
@Test
def testExp(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.exp(),
+ "f2.exp()",
"EXP(f2)",
math.exp(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.exp(),
+ "f3.exp()",
"EXP(f3)",
math.exp(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.exp(),
+ "f4.exp()",
"EXP(f4)",
math.exp(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.exp(),
+ "f5.exp()",
"EXP(f5)",
math.exp(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.exp(),
+ "f6.exp()",
"EXP(f6)",
math.exp(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f7.exp(),
+ "exp(f7)",
"EXP(f7)",
math.exp(3).toString)
- testSqlApi(
+ testAllApis(
+ 3.exp(),
+ "exp(3)",
"EXP(3)",
math.exp(3).toString)
}
@Test
def testLog10(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.log10(),
+ "f2.log10()",
"LOG10(f2)",
math.log10(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.log10(),
+ "f3.log10()",
"LOG10(f3)",
math.log10(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.log10(),
+ "f4.log10()",
"LOG10(f4)",
math.log10(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.log10(),
+ "f5.log10()",
"LOG10(f5)",
math.log10(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.log10(),
+ "f6.log10()",
"LOG10(f6)",
math.log10(4.6).toString)
}
@@ -1309,472 +1620,835 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testPower(): Unit = {
// f7: int , f4: long, f6: double
- testSqlApi(
+ testAllApis(
+ 'f2.power('f7),
+ "f2.power(f7)",
"POWER(f2, f7)",
math.pow(42.toByte, 3).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.power('f6),
+ "f3.power(f6)",
"POWER(f3, f6)",
math.pow(43.toShort, 4.6D).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.power('f5),
+ "f4.power(f5)",
"POWER(f4, f5)",
math.pow(44.toLong, 4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.power('f5),
+ "f4.power(f5)",
"POWER(f4, f5)",
math.pow(44.toLong, 4.5.toFloat).toString)
// f5: float
- testSqlApi(
+ testAllApis('f5.power('f5),
+ "f5.power(f5)",
"power(f5, f5)",
math.pow(4.5F, 4.5F).toString)
- testSqlApi(
+ testAllApis('f5.power('f6),
+ "f5.power(f6)",
"power(f5, f6)",
math.pow(4.5F, 4.6D).toString)
- testSqlApi(
+ testAllApis('f5.power('f7),
+ "f5.power(f7)",
"power(f5, f7)",
math.pow(4.5F, 3).toString)
- testSqlApi(
+ testAllApis('f5.power('f4),
+ "f5.power(f4)",
"power(f5, f4)",
math.pow(4.5F, 44L).toString)
// f22: bigDecimal
// TODO delete casting in SQL when CALCITE-1467 is fixed
- testSqlApi(
+ testAllApis(
+ 'f22.cast(DataTypes.DOUBLE).power('f5),
+ "f22.cast(DOUBLE).power(f5)",
"power(CAST(f22 AS DOUBLE), f5)",
math.pow(2, 4.5F).toString)
- testSqlApi(
+ testAllApis(
+ 'f22.cast(DataTypes.DOUBLE).power('f6),
+ "f22.cast(DOUBLE).power(f6)",
"power(CAST(f22 AS DOUBLE), f6)",
math.pow(2, 4.6D).toString)
- testSqlApi(
+ testAllApis(
+ 'f22.cast(DataTypes.DOUBLE).power('f7),
+ "f22.cast(DOUBLE).power(f7)",
"power(CAST(f22 AS DOUBLE), f7)",
math.pow(2, 3).toString)
- testSqlApi(
+ testAllApis(
+ 'f22.cast(DataTypes.DOUBLE).power('f4),
+ "f22.cast(DOUBLE).power(f4)",
"power(CAST(f22 AS DOUBLE), f4)",
math.pow(2, 44L).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.power('f22.cast(DataTypes.DOUBLE)),
+ "f6.power(f22.cast(DOUBLE))",
"power(f6, f22)",
math.pow(4.6D, 2).toString)
}
@Test
def testSqrt(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f6.sqrt(),
+ "f6.sqrt",
"SQRT(f6)",
math.sqrt(4.6D).toString)
- testSqlApi(
+ testAllApis(
+ 'f7.sqrt(),
+ "f7.sqrt",
"SQRT(f7)",
math.sqrt(3).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.sqrt(),
+ "f4.sqrt",
"SQRT(f4)",
math.sqrt(44L).toString)
- testSqlApi(
+ testAllApis(
+ 'f22.cast(DataTypes.DOUBLE).sqrt(),
+ "f22.cast(DOUBLE).sqrt",
"SQRT(CAST(f22 AS DOUBLE))",
math.sqrt(2.0).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.sqrt(),
+ "f5.sqrt",
"SQRT(f5)",
math.pow(4.5F, 0.5).toString)
- testSqlApi(
+ testAllApis(
+ 25.sqrt(),
+ "25.sqrt()",
"SQRT(25)",
"5.0")
- testSqlApi(
- // TODO fix FLINK-4621
- "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))",
+ testAllApis(
+ 2.2.sqrt(),
+ "2.2.sqrt()",
+ "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))", // TODO fix FLINK-4621
math.sqrt(2.2).toString)
}
@Test
+ def testCosh(): Unit = {
+ testAllApis(
+ 0.cosh(),
+ "0.cosh()",
+ "COSH(0)",
+ math.cosh(0).toString
+ )
+
+ testAllApis(
+ -1.cosh(),
+ "-1.cosh()",
+ "COSH(-1)",
+ math.cosh(-1).toString
+ )
+
+ testAllApis(
+ 'f4.cosh(),
+ "f4.cosh",
+ "COSH(f4)",
+ math.cosh(44L).toString)
+
+ testAllApis(
+ 'f6.cosh(),
+ "f6.cosh",
+ "COSH(f6)",
+ math.cosh(4.6D).toString)
+
+ testAllApis(
+ 'f7.cosh(),
+ "f7.cosh",
+ "COSH(f7)",
+ math.cosh(3).toString)
+
+ testAllApis(
+ 'f22.cosh(),
+ "f22.cosh",
+ "COSH(f22)",
+ math.cosh(2.0).toString)
+ }
+
+ @Test
def testLn(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.ln(),
+ "f2.ln()",
"LN(f2)",
math.log(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.ln(),
+ "f3.ln()",
"LN(f3)",
math.log(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.ln(),
+ "f4.ln()",
"LN(f4)",
math.log(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.ln(),
+ "f5.ln()",
"LN(f5)",
math.log(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.ln(),
+ "f6.ln()",
"LN(f6)",
math.log(4.6).toString)
}
@Test
def testAbs(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.abs(),
+ "f2.abs()",
"ABS(f2)",
"42")
- testSqlApi(
+ testAllApis(
+ 'f3.abs(),
+ "f3.abs()",
"ABS(f3)",
"43")
- testSqlApi(
+ testAllApis(
+ 'f4.abs(),
+ "f4.abs()",
"ABS(f4)",
"44")
- testSqlApi(
+ testAllApis(
+ 'f5.abs(),
+ "f5.abs()",
"ABS(f5)",
"4.5")
- testSqlApi(
+ testAllApis(
+ 'f6.abs(),
+ "f6.abs()",
"ABS(f6)",
"4.6")
- testSqlApi(
+ testAllApis(
+ 'f9.abs(),
+ "f9.abs()",
"ABS(f9)",
"42")
- testSqlApi(
+ testAllApis(
+ 'f10.abs(),
+ "f10.abs()",
"ABS(f10)",
"43")
- testSqlApi(
+ testAllApis(
+ 'f11.abs(),
+ "f11.abs()",
"ABS(f11)",
"44")
- testSqlApi(
+ testAllApis(
+ 'f12.abs(),
+ "f12.abs()",
"ABS(f12)",
"4.5")
- testSqlApi(
+ testAllApis(
+ 'f13.abs(),
+ "f13.abs()",
"ABS(f13)",
"4.6")
- testSqlApi(
+ testAllApis(
+ 'f15.abs(),
+ "f15.abs()",
"ABS(f15)",
"1231.1231231321321321111")
}
@Test
def testArithmeticFloorCeil(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f5.floor(),
+ "f5.floor()",
"FLOOR(f5)",
"4.0")
- testSqlApi(
+ testAllApis(
+ 'f5.ceil(),
+ "f5.ceil()",
"CEIL(f5)",
"5.0")
- testSqlApi(
+ testAllApis(
+ 'f3.floor(),
+ "f3.floor()",
"FLOOR(f3)",
"43")
- testSqlApi(
+ testAllApis(
+ 'f3.ceil(),
+ "f3.ceil()",
"CEIL(f3)",
"43")
- testSqlApi(
+ testAllApis(
+ 'f15.floor(),
+ "f15.floor()",
"FLOOR(f15)",
"-1232")
- testSqlApi(
+ testAllApis(
+ 'f15.ceil(),
+ "f15.ceil()",
"CEIL(f15)",
"-1231")
}
@Test
def testSin(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.sin(),
+ "f2.sin()",
"SIN(f2)",
math.sin(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.sin(),
+ "f3.sin()",
"SIN(f3)",
math.sin(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.sin(),
+ "f4.sin()",
"SIN(f4)",
math.sin(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.sin(),
+ "f5.sin()",
"SIN(f5)",
math.sin(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.sin(),
+ "f6.sin()",
"SIN(f6)",
math.sin(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.sin(),
+ "sin(f15)",
"SIN(f15)",
math.sin(-1231.1231231321321321111).toString)
}
@Test
def testCos(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.cos(),
+ "f2.cos()",
"COS(f2)",
math.cos(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.cos(),
+ "f3.cos()",
"COS(f3)",
math.cos(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.cos(),
+ "f4.cos()",
"COS(f4)",
math.cos(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.cos(),
+ "f5.cos()",
"COS(f5)",
math.cos(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.cos(),
+ "f6.cos()",
"COS(f6)",
math.cos(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.cos(),
+ "cos(f15)",
"COS(f15)",
math.cos(-1231.1231231321321321111).toString)
}
@Test
+ def testSinh(): Unit = {
+ testAllApis(
+ 0.sinh(),
+ "0.sinh()",
+ "SINH(0)",
+ math.sinh(0).toString)
+
+ testAllApis(
+ -1.sinh(),
+ "-1.sinh()",
+ "SINH(-1)",
+ math.sinh(-1).toString)
+
+ testAllApis(
+ 'f4.sinh(),
+ "f4.sinh",
+ "SINH(f4)",
+ math.sinh(44L).toString)
+
+ testAllApis(
+ 'f6.sinh(),
+ "f6.sinh",
+ "SINH(f6)",
+ math.sinh(4.6D).toString)
+
+ testAllApis(
+ 'f7.sinh(),
+ "f7.sinh",
+ "SINH(f7)",
+ math.sinh(3).toString)
+
+ testAllApis(
+ 'f22.sinh(),
+ "f22.sinh",
+ "SINH(f22)",
+ math.sinh(2.0).toString)
+ }
+
+ @Test
def testTan(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.tan(),
+ "f2.tan()",
"TAN(f2)",
math.tan(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.tan(),
+ "f3.tan()",
"TAN(f3)",
math.tan(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.tan(),
+ "f4.tan()",
"TAN(f4)",
math.tan(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.tan(),
+ "f5.tan()",
"TAN(f5)",
math.tan(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.tan(),
+ "f6.tan()",
"TAN(f6)",
math.tan(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.tan(),
+ "tan(f15)",
"TAN(f15)",
math.tan(-1231.1231231321321321111).toString)
}
@Test
+ def testTanh(): Unit = {
+ testAllApis(
+ 0.tanh(),
+ "0.tanh()",
+ "TANH(0)",
+ math.tanh(0).toString)
+
+ testAllApis(
+ -1.tanh(),
+ "-1.tanh()",
+ "TANH(-1)",
+ math.tanh(-1).toString)
+
+ testAllApis(
+ 'f4.tanh(),
+ "f4.tanh",
+ "TANH(f4)",
+ math.tanh(44L).toString)
+
+ testAllApis(
+ 'f6.tanh(),
+ "f6.tanh",
+ "TANH(f6)",
+ math.tanh(4.6D).toString)
+
+ testAllApis(
+ 'f7.tanh(),
+ "f7.tanh",
+ "TANH(f7)",
+ math.tanh(3).toString)
+
+ testAllApis(
+ 'f22.tanh(),
+ "f22.tanh",
+ "TANH(f22)",
+ math.tanh(2.0).toString)
+ }
+
+ @Test
def testCot(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.cot(),
+ "f2.cot()",
"COT(f2)",
(1.0d / math.tan(42.toByte)).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.cot(),
+ "f3.cot()",
"COT(f3)",
(1.0d / math.tan(43.toShort)).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.cot(),
+ "f4.cot()",
"COT(f4)",
(1.0d / math.tan(44.toLong)).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.cot(),
+ "f5.cot()",
"COT(f5)",
(1.0d / math.tan(4.5.toFloat)).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.cot(),
+ "f6.cot()",
"COT(f6)",
(1.0d / math.tan(4.6)).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.cot(),
+ "cot(f15)",
"COT(f15)",
(1.0d / math.tan(-1231.1231231321321321111)).toString)
}
@Test
def testAsin(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f25.asin(),
+ "f25.asin()",
"ASIN(f25)",
math.asin(0.42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f26.asin(),
+ "f26.asin()",
"ASIN(f26)",
math.asin(0.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f27.asin(),
+ "f27.asin()",
"ASIN(f27)",
math.asin(0.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f28.asin(),
+ "f28.asin()",
"ASIN(f28)",
math.asin(0.45.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f29.asin(),
+ "f29.asin()",
"ASIN(f29)",
math.asin(0.46).toString)
- testSqlApi(
+ testAllApis(
+ 'f30.asin(),
+ "f30.asin()",
"ASIN(f30)",
math.asin(1).toString)
- testSqlApi(
+ testAllApis(
+ 'f31.asin(),
+ "f31.asin()",
"ASIN(f31)",
math.asin(-0.1231231321321321111).toString)
}
@Test
def testAcos(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f25.acos(),
+ "f25.acos()",
"ACOS(f25)",
math.acos(0.42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f26.acos(),
+ "f26.acos()",
"ACOS(f26)",
math.acos(0.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f27.acos(),
+ "f27.acos()",
"ACOS(f27)",
math.acos(0.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f28.acos(),
+ "f28.acos()",
"ACOS(f28)",
math.acos(0.45.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f29.acos(),
+ "f29.acos()",
"ACOS(f29)",
math.acos(0.46).toString)
- testSqlApi(
+ testAllApis(
+ 'f30.acos(),
+ "f30.acos()",
"ACOS(f30)",
math.acos(1).toString)
- testSqlApi(
+ testAllApis(
+ 'f31.acos(),
+ "f31.acos()",
"ACOS(f31)",
math.acos(-0.1231231321321321111).toString)
}
@Test
def testAtan(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f25.atan(),
+ "f25.atan()",
"ATAN(f25)",
math.atan(0.42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f26.atan(),
+ "f26.atan()",
"ATAN(f26)",
math.atan(0.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f27.atan(),
+ "f27.atan()",
"ATAN(f27)",
math.atan(0.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f28.atan(),
+ "f28.atan()",
"ATAN(f28)",
math.atan(0.45.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f29.atan(),
+ "f29.atan()",
"ATAN(f29)",
math.atan(0.46).toString)
- testSqlApi(
+ testAllApis(
+ 'f30.atan(),
+ "f30.atan()",
"ATAN(f30)",
math.atan(1).toString)
- testSqlApi(
+ testAllApis(
+ 'f31.atan(),
+ "f31.atan()",
"ATAN(f31)",
math.atan(-0.1231231321321321111).toString)
}
@Test
+ def testAtan2(): Unit = {
+ testAllApis(
+ atan2('f25, 'f26),
+ "atan2(f25, f26)",
+ "ATAN2(f25, f26)",
+ math.atan2(0.42.toByte, 0.toByte).toString)
+
+ testAllApis(
+ atan2('f26, 'f25),
+ "atan2(f26, f25)",
+ "ATAN2(f26, f25)",
+ math.atan2(0.toShort, 0.toShort).toString)
+
+ testAllApis(
+ atan2('f27, 'f27),
+ "atan2(f27, f27)",
+ "ATAN2(f27, f27)",
+ math.atan2(0.toLong, 0.toLong).toString)
+
+ testAllApis(
+ atan2('f28, 'f28),
+ "atan2(f28, f28)",
+ "ATAN2(f28, f28)",
+ math.atan2(0.45.toFloat, 0.45.toFloat).toString)
+
+ testAllApis(
+ atan2('f29, 'f29),
+ "atan2(f29, f29)",
+ "ATAN2(f29, f29)",
+ math.atan2(0.46, 0.46).toString)
+
+ testAllApis(
+ atan2('f30, 'f30),
+ "atan2(f30, f30)",
+ "ATAN2(f30, f30)",
+ math.atan2(1, 1).toString)
+
+ testAllApis(
+ atan2('f31, 'f31),
+ "atan2(f31, f31)",
+ "ATAN2(f31, f31)",
+ math.atan2(-0.1231231321321321111, -0.1231231321321321111).toString)
+ }
+
+ @Test
def testDegrees(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.degrees(),
+ "f2.degrees()",
"DEGREES(f2)",
math.toDegrees(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.degrees(),
+ "f3.degrees()",
"DEGREES(f3)",
math.toDegrees(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.degrees(),
+ "f4.degrees()",
"DEGREES(f4)",
math.toDegrees(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.degrees(),
+ "f5.degrees()",
"DEGREES(f5)",
math.toDegrees(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.degrees(),
+ "f6.degrees()",
"DEGREES(f6)",
math.toDegrees(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.degrees(),
+ "degrees(f15)",
"DEGREES(f15)",
math.toDegrees(-1231.1231231321321321111).toString)
}
@Test
def testRadians(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f2.radians(),
+ "f2.radians()",
"RADIANS(f2)",
math.toRadians(42.toByte).toString)
- testSqlApi(
+ testAllApis(
+ 'f3.radians(),
+ "f3.radians()",
"RADIANS(f3)",
math.toRadians(43.toShort).toString)
- testSqlApi(
+ testAllApis(
+ 'f4.radians(),
+ "f4.radians()",
"RADIANS(f4)",
math.toRadians(44.toLong).toString)
- testSqlApi(
+ testAllApis(
+ 'f5.radians(),
+ "f5.radians()",
"RADIANS(f5)",
math.toRadians(4.5.toFloat).toString)
- testSqlApi(
+ testAllApis(
+ 'f6.radians(),
+ "f6.radians()",
"RADIANS(f6)",
math.toRadians(4.6).toString)
- testSqlApi(
+ testAllApis(
+ 'f15.radians(),
+ "radians(f15)",
"RADIANS(f15)",
math.toRadians(-1231.1231231321321321111).toString)
}
@Test
def testSign(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f4.sign(),
+ "f4.sign()",
"SIGN(f4)",
1.toString)
- testSqlApi(
+ testAllApis(
+ 'f6.sign(),
+ "f6.sign()",
"SIGN(f6)",
1.0.toString)
- testSqlApi(
+ testAllApis(
+ 'f15.sign(),
+ "sign(f15)",
"SIGN(f15)",
"-1.0000000000000000000") // calcite: SIGN(Decimal(p,s)) => Decimal(p,s)
}
@Test
def testRound(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f29.round('f30),
+ "f29.round(f30)",
"ROUND(f29, f30)",
0.5.toString)
- testSqlApi(
- "ROUND(f31, f7)",
- "-0.123")
-
- testSqlApi(
+ testAllApis(
+ 'f4.round('f32),
+ "f4.round(f32)",
"ROUND(f4, f32)",
40.toString)
@@ -1797,74 +2471,92 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testPi(): Unit = {
- // PI function
- testSqlApi(
- "PI()",
- math.Pi.toString)
-
- // PI operator
- testSqlApi(
- "PI",
+ testAllApis(
+ pi(),
+ "pi()",
+ "pi()",
math.Pi.toString)
}
@Test
def testRandAndRandInteger(): Unit = {
val random1 = new java.util.Random(1)
- testSqlApi(
+ testAllApis(
+ rand(1),
+ "rand(1)",
"RAND(1)",
random1.nextDouble().toString)
val random2 = new java.util.Random(3)
- testSqlApi(
+ testAllApis(
+ rand('f7),
+ "rand(f7)",
"RAND(f7)",
random2.nextDouble().toString)
val random3 = new java.util.Random(1)
- testSqlApi(
+ testAllApis(
+ randInteger(1, 10),
+ "randInteger(1, 10)",
"RAND_INTEGER(1, 10)",
random3.nextInt(10).toString)
val random4 = new java.util.Random(3)
- testSqlApi(
+ testAllApis(
+ randInteger('f7, 'f4.cast(DataTypes.INT)),
+ "randInteger(f7, f4.cast(INT))",
"RAND_INTEGER(f7, CAST(f4 AS INT))",
random4.nextInt(44).toString)
}
@Test
def testE(): Unit = {
- testSqlApi(
+ testAllApis(
+ e(),
+ "E()",
"E()",
math.E.toString)
- testSqlApi(
+ testAllApis(
+ e(),
+ "e()",
"e()",
math.E.toString)
}
@Test
def testLog(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f6.log(),
+ "f6.log",
"LOG(f6)",
"1.5260563034950492"
)
- testSqlApi(
+ testAllApis(
+ ('f6 - 'f6 + 100).log('f6 - 'f6 + 10),
+ "(f6 - f6 + 100).log(f6 - f6 + 10)",
"LOG(f6 - f6 + 10, f6 - f6 + 100)",
"2.0"
)
- testSqlApi(
+ testAllApis(
+ ('f6 + 20).log(),
+ "(f6+20).log",
"LOG(f6+20)",
"3.202746442938317"
)
- testSqlApi(
+ testAllApis(
+ 10.log(),
+ "10.log",
"LOG(10)",
"2.302585092994046"
)
- testSqlApi(
+ testAllApis(
+ 100.log(10),
+ "100.log(10)",
"LOG(10, 100)",
"2.0"
)
@@ -1891,17 +2583,28 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testLog2(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f6.log2(),
+ "f6.log2",
"LOG2(f6)",
- "2.2016338611696504"
- )
+ "2.2016338611696504")
- testSqlApi(
+ testAllApis(
+ ('f6 - 'f6 + 100).log2(),
+ "(f6 - f6 + 100).log2()",
+ "LOG2(f6 - f6 + 100)",
+ "6.643856189774725")
+
+ testAllApis(
+ ('f6 + 20).log2(),
+ "(f6+20).log2",
"LOG2(f6+20)",
"4.620586410451877"
)
- testSqlApi(
+ testAllApis(
+ 10.log2(),
+ "10.log2",
"LOG2(10)",
"3.3219280948873626"
)
@@ -1981,75 +2684,147 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testExtract(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.YEAR),
+ "f16.extract(YEAR)",
"EXTRACT(YEAR FROM f16)",
"1996")
- testSqlApi(
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.QUARTER),
+ "f16.extract(QUARTER)",
+ "EXTRACT(QUARTER FROM f16)",
+ "4")
+
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.MONTH),
+ "extract(f16, MONTH)",
"EXTRACT(MONTH FROM f16)",
"11")
- testSqlApi(
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.WEEK),
+ "extract(f16, WEEK)",
+ "EXTRACT(WEEK FROM f16)",
+ "45")
+
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.DAY),
+ "f16.extract(DAY)",
"EXTRACT(DAY FROM f16)",
"10")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.YEAR),
+ "f18.extract(YEAR)",
"EXTRACT(YEAR FROM f18)",
"1996")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.QUARTER),
+ "f18.extract(QUARTER)",
+ "EXTRACT(QUARTER FROM f18)",
+ "4")
+
+ testAllApis(
+ 'f16.extract(TimeIntervalUnit.QUARTER),
+ "f16.extract(QUARTER)",
+ "EXTRACT(QUARTER FROM f16)",
+ "4")
+
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.MONTH),
+ "f18.extract(MONTH)",
"EXTRACT(MONTH FROM f18)",
"11")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.WEEK),
+ "f18.extract(WEEK)",
+ "EXTRACT(WEEK FROM f18)",
+ "45")
+
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.DAY),
+ "f18.extract(DAY)",
"EXTRACT(DAY FROM f18)",
"10")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.HOUR),
+ "f18.extract(HOUR)",
"EXTRACT(HOUR FROM f18)",
"6")
- testSqlApi(
+ testAllApis(
+ 'f17.extract(TimeIntervalUnit.HOUR),
+ "f17.extract(HOUR)",
"EXTRACT(HOUR FROM f17)",
"6")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.MINUTE),
+ "f18.extract(MINUTE)",
"EXTRACT(MINUTE FROM f18)",
"55")
- testSqlApi(
+ testAllApis(
+ 'f17.extract(TimeIntervalUnit.MINUTE),
+ "f17.extract(MINUTE)",
"EXTRACT(MINUTE FROM f17)",
"55")
- testSqlApi(
+ testAllApis(
+ 'f18.extract(TimeIntervalUnit.SECOND),
+ "f18.extract(SECOND)",
"EXTRACT(SECOND FROM f18)",
"44")
- testSqlApi(
+ testAllApis(
+ 'f17.extract(TimeIntervalUnit.SECOND),
+ "f17.extract(SECOND)",
"EXTRACT(SECOND FROM f17)",
"44")
- testSqlApi(
+ testAllApis(
+ 'f19.extract(TimeIntervalUnit.DAY),
+ "f19.extract(DAY)",
"EXTRACT(DAY FROM f19)",
"16979")
- testSqlApi(
+ testAllApis(
+ 'f19.extract(TimeIntervalUnit.HOUR),
+ "f19.extract(HOUR)",
"EXTRACT(HOUR FROM f19)",
"7")
- testSqlApi(
+ testAllApis(
+ 'f19.extract(TimeIntervalUnit.MINUTE),
+ "f19.extract(MINUTE)",
"EXTRACT(MINUTE FROM f19)",
"23")
- testSqlApi(
+ testAllApis(
+ 'f19.extract(TimeIntervalUnit.SECOND),
+ "f19.extract(SECOND)",
"EXTRACT(SECOND FROM f19)",
"33")
- testSqlApi(
+ testAllApis(
+ 'f20.extract(TimeIntervalUnit.MONTH),
+ "f20.extract(MONTH)",
"EXTRACT(MONTH FROM f20)",
"1")
- testSqlApi(
+ testAllApis(
+ 'f20.extract(TimeIntervalUnit.QUARTER),
+ "f20.extract(QUARTER)",
+ "EXTRACT(QUARTER FROM f20)",
+ "1")
+
+ testAllApis(
+ 'f20.extract(TimeIntervalUnit.YEAR),
+ "f20.extract(YEAR)",
"EXTRACT(YEAR FROM f20)",
"2")
@@ -2185,83 +2960,123 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testTemporalFloor(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f18.floor(TimeIntervalUnit.YEAR),
+ "f18.floor(YEAR)",
"FLOOR(f18 TO YEAR)",
"1996-01-01 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.floor(TimeIntervalUnit.MONTH),
+ "f18.floor(MONTH)",
"FLOOR(f18 TO MONTH)",
"1996-11-01 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.floor(TimeIntervalUnit.DAY),
+ "f18.floor(DAY)",
"FLOOR(f18 TO DAY)",
"1996-11-10 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.floor(TimeIntervalUnit.MINUTE),
+ "f18.floor(MINUTE)",
"FLOOR(f18 TO MINUTE)",
"1996-11-10 06:55:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.floor(TimeIntervalUnit.SECOND),
+ "f18.floor(SECOND)",
"FLOOR(f18 TO SECOND)",
"1996-11-10 06:55:44.000")
- testSqlApi(
+ testAllApis(
+ 'f17.floor(TimeIntervalUnit.HOUR),
+ "f17.floor(HOUR)",
"FLOOR(f17 TO HOUR)",
"06:00:00")
- testSqlApi(
+ testAllApis(
+ 'f17.floor(TimeIntervalUnit.MINUTE),
+ "f17.floor(MINUTE)",
"FLOOR(f17 TO MINUTE)",
"06:55:00")
- testSqlApi(
+ testAllApis(
+ 'f17.floor(TimeIntervalUnit.SECOND),
+ "f17.floor(SECOND)",
"FLOOR(f17 TO SECOND)",
"06:55:44")
- testSqlApi(
+ testAllApis(
+ 'f16.floor(TimeIntervalUnit.YEAR),
+ "f16.floor(YEAR)",
"FLOOR(f16 TO YEAR)",
"1996-01-01")
- testSqlApi(
+ testAllApis(
+ 'f16.floor(TimeIntervalUnit.MONTH),
+ "f16.floor(MONTH)",
"FLOOR(f16 TO MONTH)",
"1996-11-01")
- testSqlApi(
+ testAllApis(
+ 'f18.ceil(TimeIntervalUnit.YEAR),
+ "f18.ceil(YEAR)",
"CEIL(f18 TO YEAR)",
"1997-01-01 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.ceil(TimeIntervalUnit.MONTH),
+ "f18.ceil(MONTH)",
"CEIL(f18 TO MONTH)",
"1996-12-01 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.ceil(TimeIntervalUnit.DAY),
+ "f18.ceil(DAY)",
"CEIL(f18 TO DAY)",
"1996-11-11 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.ceil(TimeIntervalUnit.MINUTE),
+ "f18.ceil(MINUTE)",
"CEIL(f18 TO MINUTE)",
"1996-11-10 06:56:00.000")
- testSqlApi(
+ testAllApis(
+ 'f18.ceil(TimeIntervalUnit.SECOND),
+ "f18.ceil(SECOND)",
"CEIL(f18 TO SECOND)",
"1996-11-10 06:55:45.000")
- testSqlApi(
+ testAllApis(
+ 'f17.ceil(TimeIntervalUnit.HOUR),
+ "f17.ceil(HOUR)",
"CEIL(f17 TO HOUR)",
"07:00:00")
- testSqlApi(
+ testAllApis(
+ 'f17.ceil(TimeIntervalUnit.MINUTE),
+ "f17.ceil(MINUTE)",
"CEIL(f17 TO MINUTE)",
"06:56:00")
- testSqlApi(
+ testAllApis(
+ 'f17.ceil(TimeIntervalUnit.SECOND),
+ "f17.ceil(SECOND)",
"CEIL(f17 TO SECOND)",
"06:55:44")
- testSqlApi(
+ testAllApis(
+ 'f16.ceil(TimeIntervalUnit.YEAR),
+ "f16.ceil(YEAR)",
"CEIL(f16 TO YEAR)",
"1997-01-01")
- testSqlApi(
+ testAllApis(
+ 'f16.ceil(TimeIntervalUnit.MONTH),
+ "f16.ceil(MONTH)",
"CEIL(f16 TO MONTH)",
"1996-12-01")
}
@@ -2273,58 +3088,86 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
// we just test the format of the output
// manual test can be found in NonDeterministicTests
- testSqlApi(
+ testAllApis(
+ currentDate().cast(DataTypes.STRING).charLength() >= 5,
+ "currentDate().cast(STRING).charLength() >= 5",
"CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR)) >= 5",
"true")
- testSqlApi(
+ testAllApis(
+ currentTime().cast(DataTypes.STRING).charLength() >= 5,
+ "currentTime().cast(STRING).charLength() >= 5",
"CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR)) >= 5",
"true")
- testSqlApi(
+ testAllApis(
+ currentTimestamp().cast(DataTypes.STRING).charLength() >= 12,
+ "currentTimestamp().cast(STRING).charLength() >= 12",
"CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 12",
"true")
- testSqlApi(
+ testAllApis(
+ localTimestamp().cast(DataTypes.STRING).charLength() >= 12,
+ "localTimestamp().cast(STRING).charLength() >= 12",
"CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 12",
"true")
- testSqlApi(
+ testAllApis(
+ localTime().cast(DataTypes.STRING).charLength() >= 5,
+ "localTime().cast(STRING).charLength() >= 5",
"CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR)) >= 5",
"true")
// comparisons are deterministic
- testSqlApi(
+ testAllApis(
+ localTimestamp() === localTimestamp(),
+ "localTimestamp() === localTimestamp()",
"LOCALTIMESTAMP = LOCALTIMESTAMP",
"true")
}
@Test
def testOverlaps(): Unit = {
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hours),
+ "temporalOverlaps('2:55:00'.toTime, 1.hour, '3:30:00'.toTime, 2.hours)",
"(TIME '2:55:00', INTERVAL '1' HOUR) OVERLAPS (TIME '3:30:00', INTERVAL '2' HOUR)",
"true")
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("9:00:00".toTime, "9:30:00".toTime, "9:29:00".toTime, "9:31:00".toTime),
+ "temporalOverlaps(toTime('9:00:00'), '9:30:00'.toTime, '9:29:00'.toTime, '9:31:00'.toTime)",
"(TIME '9:00:00', TIME '9:30:00') OVERLAPS (TIME '9:29:00', TIME '9:31:00')",
"true")
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("9:00:00".toTime, "10:00:00".toTime, "10:15:00".toTime, 3.hours),
+ "temporalOverlaps('9:00:00'.toTime, '10:00:00'.toTime, '10:15:00'.toTime, 3.hours)",
"(TIME '9:00:00', TIME '10:00:00') OVERLAPS (TIME '10:15:00', INTERVAL '3' HOUR)",
"false")
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("2011-03-10".toDate, 10.days, "2011-03-19".toDate, 10.days),
+ "temporalOverlaps(toDate('2011-03-10'), 10.days, '2011-03-19'.toDate, 10.days)",
"(DATE '2011-03-10', INTERVAL '10' DAY) OVERLAPS (DATE '2011-03-19', INTERVAL '10' DAY)",
"true")
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("2011-03-10 05:02:02".toTimestamp, 0.milli,
+ "2011-03-10 05:02:02".toTimestamp, "2011-03-10 05:02:01".toTimestamp),
+ "temporalOverlaps(toTimestamp('2011-03-10 05:02:02'), 0.milli, " +
+ "'2011-03-10 05:02:02'.toTimestamp, '2011-03-10 05:02:01'.toTimestamp)",
"(TIMESTAMP '2011-03-10 05:02:02', INTERVAL '0' SECOND) OVERLAPS " +
- "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')",
+ "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')",
"true")
- testSqlApi(
+ testAllApis(
+ temporalOverlaps("2011-03-10 02:02:02.001".toTimestamp, 0.milli,
+ "2011-03-10 02:02:02.002".toTimestamp, "2011-03-10 02:02:02.002".toTimestamp),
+ "temporalOverlaps('2011-03-10 02:02:02.001'.toTimestamp, 0.milli, " +
+ "'2011-03-10 02:02:02.002'.toTimestamp, '2011-03-10 02:02:02.002'.toTimestamp)",
"(TIMESTAMP '2011-03-10 02:02:02.001', INTERVAL '0' SECOND) OVERLAPS " +
- "(TIMESTAMP '2011-03-10 02:02:02.002', TIMESTAMP '2011-03-10 02:02:02.002')",
+ "(TIMESTAMP '2011-03-10 02:02:02.002', TIMESTAMP '2011-03-10 02:02:02.002')",
"false")
}
@@ -2394,7 +3237,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
for ((data,index) <- dataParts.zipWithIndex) {
index match {
case 0 => // timestamp, timestamp
- testSqlApi(
+ testAllApis(
+ timestampDiff(unitParts._2, data._1.toTimestamp, data._2.toTimestamp),
+ s"timestampDiff(${unitParts._1}, '${data._1}'.toTimestamp, '${data._2}'.toTimestamp)",
s"TIMESTAMPDIFF(${unitParts._1}, TIMESTAMP '${data._1}', TIMESTAMP '${data._2}')",
data._3
)
@@ -2403,7 +3248,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
data._3
)
case 1 => // date, timestamp
- testSqlApi(
+ testAllApis(
+ timestampDiff(unitParts._2, data._1.toDate, data._2.toTimestamp),
+ s"timestampDiff(${unitParts._1}, '${data._1}'.toDate, '${data._2}'.toTimestamp)",
s"TIMESTAMPDIFF(${unitParts._1}, DATE '${data._1}', TIMESTAMP '${data._2}')",
data._3
)
@@ -2412,7 +3259,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
data._3
)
case 2 => // timestamp, date
- testSqlApi(
+ testAllApis(
+ timestampDiff(unitParts._2, data._1.toTimestamp, data._2.toDate),
+ s"timestampDiff(${unitParts._1}, '${data._1}'.toTimestamp, '${data._2}'.toDate)",
s"TIMESTAMPDIFF(${unitParts._1}, TIMESTAMP '${data._1}', DATE '${data._2}')",
data._3
)
@@ -2421,7 +3270,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
data._3
)
case 3 => // date, date
- testSqlApi(
+ testAllApis(
+ timestampDiff(unitParts._2, data._1.toDate, data._2.toDate),
+ s"timestampDiff(${unitParts._1}, '${data._1}'.toDate, '${data._2}'.toDate)",
s"TIMESTAMPDIFF(${unitParts._1}, DATE '${data._1}', DATE '${data._2}')",
data._3
)
@@ -2433,12 +3284,18 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
}
}
- testSqlApi(
+ testAllApis(
+ timestampDiff(TimePointUnit.DAY, nullOf(DataTypes.TIMESTAMP(3)),
+ "2016-02-24 12:42:25".toTimestamp),
+ "timestampDiff(DAY, Null(SQL_TIMESTAMP), '2016-02-24 12:42:25'.toTimestamp)",
"TIMESTAMPDIFF(DAY, CAST(NULL AS TIMESTAMP), TIMESTAMP '2016-02-24 12:42:25')",
"null"
)
- testSqlApi(
+ testAllApis(
+ timestampDiff(TimePointUnit.DAY, "2016-02-24 12:42:25".toTimestamp,
+ nullOf(DataTypes.TIMESTAMP(3))),
+ "timestampDiff(DAY, '2016-02-24 12:42:25'.toTimestamp, Null(SQL_TIMESTAMP))",
"TIMESTAMPDIFF(DAY, TIMESTAMP '2016-02-24 12:42:25', CAST(NULL AS TIMESTAMP))",
"null"
)
@@ -2448,11 +3305,11 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
def testTimestampAdd(): Unit = {
val data = Seq(
- (1, "TIMESTAMP '2017-11-29 22:58:58.998'"),
- (3, "TIMESTAMP '2017-11-29 22:58:58.998'"),
- (-1, "TIMESTAMP '2017-11-29 22:58:58.998'"),
- (-61, "TIMESTAMP '2017-11-29 22:58:58.998'"),
- (-1000, "TIMESTAMP '2017-11-29 22:58:58.998'")
+ (1, "2017-11-29 22:58:58.998"),
+ (3, "2017-11-29 22:58:58.998"),
+ (-1, "2017-11-29 22:58:58.998"),
+ (-61, "2017-11-29 22:58:58.998"),
+ (-1000, "2017-11-29 22:58:58.998")
)
val YEAR = Seq(
@@ -2531,27 +3388,141 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
"SQL_TSI_SECOND" -> SECOND
)
+ def intervalCount(interval: String, count: Int): (Expression, String) = interval match {
+ case "YEAR" => (count.years, s"$count.years")
+ case "SQL_TSI_YEAR" => (count.years, s"$count.years")
+ case "QUARTER" => (count.quarters, s"$count.quarters")
+ case "SQL_TSI_QUARTER" => (count.quarters, s"$count.quarters")
+ case "MONTH" => (count.months, s"$count.months")
+ case "SQL_TSI_MONTH" => (count.months, s"$count.months")
+ case "WEEK" => (count.weeks, s"$count.weeks")
+ case "SQL_TSI_WEEK" => (count.weeks, s"$count.weeks")
+ case "DAY" => (count.days, s"$count.days")
+ case "SQL_TSI_DAY" => (count.days, s"$count.days")
+ case "HOUR" => (count.hours, s"$count.hours")
+ case "SQL_TSI_HOUR" => (count.hours, s"$count.hours")
+ case "MINUTE" => (count.minutes, s"$count.minutes")
+ case "SQL_TSI_MINUTE" => (count.minutes, s"$count.minutes")
+ case "SECOND" => (count.seconds, s"$count.seconds")
+ case "SQL_TSI_SECOND" => (count.seconds, s"$count.seconds")
+ }
+
for ((interval, result) <- intervalMapResults) {
- testSqlApi(
- s"TIMESTAMPADD($interval, ${data.head._1}, ${data.head._2})", result.head)
- testSqlApi(
- s"TIMESTAMPADD($interval, ${data(1)._1}, ${data(1)._2})", result(1))
- testSqlApi(
- s"TIMESTAMPADD($interval, ${data(2)._1}, ${data(2)._2})", result(2))
- testSqlApi(
- s"TIMESTAMPADD($interval, ${data(3)._1}, ${data(3)._2})", result(3))
- testSqlApi(
- s"TIMESTAMPADD($interval, ${data(4)._1}, ${data(4)._2})", result(4))
+ for (i <- 0 to 4) {
+ val (offset, ts) = data(i)
+ val timeInterval = intervalCount(interval, offset)
+ testAllApis(
+ timeInterval._1 + ts.toTimestamp,
+ s"${timeInterval._2} + '$ts'.toTimestamp",
+ s"TIMESTAMPADD($interval, $offset, TIMESTAMP '$ts')",
+ result(i))
+ }
}
- testSqlApi("TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')", "null")
+ testAllApis(
+ "2016-02-24 12:42:25".toTimestamp + nullOf(DataTypes.INTERVAL(DataTypes.MINUTE())),
+ "'2016-02-24 12:42:25'.toTimestamp + Null(INTERVAL_MILLIS)",
+ "TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')",
+ "null")
- testSqlApi("TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))", "null")
+ testAllApis(
+ nullOf(DataTypes.TIMESTAMP(3)) + -200.hours,
+ "Null(SQL_TIMESTAMP) + -200.hours",
+ "TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))",
+ "null")
- testSqlApi("TIMESTAMPADD(DAY, 1, DATE '2016-06-15')", "2016-06-16")
+ testAllApis(
+ "2016-06-15".toDate + 1.day,
+ "'2016-06-15'.toDate + 1.day",
+ "TIMESTAMPADD(DAY, 1, DATE '2016-06-15')",
+ "2016-06-16")
- testSqlApi("TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))", "null")
+ testAllApis(
+ nullOf(DataTypes.TIMESTAMP(3)) + 3.months,
+ "Null(SQL_TIMESTAMP) + 3.months",
+ "TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))",
+ "null")
+
+ testAllApis(
+ "2016-02-24 12:42:25".toTimestamp + nullOf(Types.INTERVAL_MILLIS),
+ "'2016-02-24 12:42:25'.toTimestamp + nullOf(INTERVAL_MILLIS)",
+ "TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')",
+ "null")
+
+ testAllApis(
+ nullOf(Types.SQL_TIMESTAMP) + -200.hours,
+ "nullOf(SQL_TIMESTAMP) + -200.hours",
+ "TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))",
+ "null")
+
+ testAllApis(
+ nullOf(Types.SQL_TIMESTAMP) + 3.months,
+ "nullOf(SQL_TIMESTAMP) + 3.months",
+ "TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))",
+ "null")
+ // TIMESTAMPADD with DATE returns a TIMESTAMP value for sub-day intervals.
+ testAllApis("2016-06-15".toDate + 1.month,
+ "'2016-06-15'.toDate + 1.month",
+ "timestampadd(MONTH, 1, date '2016-06-15')",
+ "2016-07-15")
+
+ testAllApis("2016-06-15".toDate + 1.day,
+ "'2016-06-15'.toDate + 1.day",
+ "timestampadd(DAY, 1, date '2016-06-15')",
+ "2016-06-16")
+
+ // TODO support '2016-06-15'.toTimestamp
+// testAllApis("2016-06-15".toTimestamp - 1.hour,
+// "'2016-06-15'.toTimestamp - 1.hour",
+// "timestampadd(HOUR, -1, date '2016-06-15')",
+// "2016-06-14 23:00:00.0")
+
+// testAllApis("2016-06-15".toTimestamp + 1.minute,
+// "'2016-06-15'.toTimestamp + 1.minute",
+// "timestampadd(MINUTE, 1, date '2016-06-15')",
+// "2016-06-15 00:01:00.0")
+
+// testAllApis("2016-06-15".toTimestamp - 1.second,
+// "'2016-06-15'.toTimestamp - 1.second",
+// "timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')",
+// "2016-06-14 23:59:59.0")
+
+// testAllApis("2016-06-15".toTimestamp + 1.second,
+// "'2016-06-15'.toTimestamp + 1.second",
+// "timestampadd(SECOND, 1, date '2016-06-15')",
+// "2016-06-15 00:00:01.0")
+
+ testAllApis(nullOf(Types.SQL_TIMESTAMP) + 1.second,
+ "nullOf(SQL_TIMESTAMP) + 1.second",
+ "timestampadd(SECOND, 1, cast(null as date))",
+ "null")
+
+ testAllApis(nullOf(Types.SQL_TIMESTAMP) + 1.day,
+ "nullOf(SQL_TIMESTAMP) + 1.day",
+ "timestampadd(DAY, 1, cast(null as date))",
+ "null")
+
+ // Round to the last day of previous month
+ testAllApis("2016-05-31".toDate + 1.month,
+ "'2016-05-31'.toDate + 1.month",
+ "timestampadd(MONTH, 1, date '2016-05-31')",
+ "2016-06-30")
+
+ testAllApis("2016-01-31".toDate + 5.month,
+ "'2016-01-31'.toDate + 5.month",
+ "timestampadd(MONTH, 5, date '2016-01-31')",
+ "2016-06-30")
+
+ testAllApis("2016-03-31".toDate - 1.month,
+ "'2016-03-31'.toDate - 1.month",
+ "timestampadd(MONTH, -1, date '2016-03-31')",
+ "2016-02-29")
+
+ testAllApis("2016-03-31".toDate - 1.week,
+ "'2016-03-31'.toDate - 1.week",
+ "timestampadd(WEEK, -1, date '2016-03-31')",
+ "2016-03-24")
}
@Test
@@ -2602,93 +3573,131 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
val expectedSha224 = "90a3ed9e32b2aaf4c61c410eb925426119e1a9dc53d4286ade99a809"
val expectedSha256 = "9f86d081884c7d659a2feaa0c55ad015a3bf4f1b2b0b822cd15d6c15b0f00a08"
val expectedSha384 = "768412320f7b0aa5812fce428dc4706b3cae50e02a64caa16a7" +
- "82249bfe8efc4b7ef1ccb126255d196047dfedf17a0a9"
+ "82249bfe8efc4b7ef1ccb126255d196047dfedf17a0a9"
val expectedSha512 = "ee26b0dd4af7e749aa1a8ee3c10ae9923f618980772e473f8819a" +
- "5d4940e0db27ac185f8a0e1d5f84f88bc887fd67b143732c304cc5fa9ad8e6f57f50028a8ff"
+ "5d4940e0db27ac185f8a0e1d5f84f88bc887fd67b143732c304cc5fa9ad8e6f57f50028a8ff"
- testSqlApi(
+ testAllApis(
+ "test".md5(),
+ "md5('test')",
"MD5('test')",
expectedMd5)
- testSqlApi(
+ testAllApis(
+ "test".sha1(),
+ "sha1('test')",
"SHA1('test')",
expectedSha1)
// sha224
- testSqlApi(
+ testAllApis(
+ "test".sha224(),
+ "sha224('test')",
"SHA224('test')",
expectedSha224)
// sha-2 224
- testSqlApi(
+ testAllApis(
+ "test".sha2(224),
+ "sha2('test', 224)",
"SHA2('test', 224)",
expectedSha224)
// sha256
- testSqlApi(
+ testAllApis(
+ "test".sha256(),
+ "sha256('test')",
"SHA256('test')",
expectedSha256)
// sha-2 256
- testSqlApi(
+ testAllApis(
+ "test".sha2(256),
+ "sha2('test', 256)",
"SHA2('test', 256)",
expectedSha256)
// sha384
- testSqlApi(
+ testAllApis(
+ "test".sha384(),
+ "sha384('test')",
"SHA384('test')",
expectedSha384)
// sha-2 384
- testSqlApi(
+ testAllApis(
+ "test".sha2(384),
+ "sha2('test', 384)",
"SHA2('test', 384)",
expectedSha384)
// sha512
- testSqlApi(
+ testAllApis(
+ "test".sha512(),
+ "sha512('test')",
"SHA512('test')",
expectedSha512)
// sha-2 512
- testSqlApi(
+ testAllApis(
+ "test".sha2(512),
+ "sha2('test', 512)",
"SHA2('test', 512)",
expectedSha512)
// null tests
- testSqlApi(
+ testAllApis(
+ 'f33.md5(),
+ "md5(f33)",
"MD5(f33)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha1(),
+ "sha1(f33)",
"SHA1(f33)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha224(),
+ "sha224(f33)",
"SHA2(f33, 224)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha2(224),
+ "sha2(f33, 224)",
"SHA2(f33, 224)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha256(),
+ "sha256(f33)",
"SHA2(f33, 256)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha384(),
+ "sha384(f33)",
"SHA2(f33, 384)",
"null")
- testSqlApi(
+ testAllApis(
+ 'f33.sha512(),
+ "sha512(f33)",
"SHA2(f33, 512)",
"null")
- testSqlApi(
+ testAllApis(
+ "test".sha2(nullOf(DataTypes.INT)),
+ "sha2('test', Null(INT))",
"SHA2('test', CAST(NULL AS INT))",
"null")
// non-constant bit length
- testSqlApi(
+ testAllApis(
+ "test".sha2('f44),
+ "sha2('test', f44)",
"SHA2('test', f44)",
expectedSha256)
}
@@ -2699,35 +3708,51 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testIsTrueIsFalse(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f1.isTrue,
+ "f1.isTrue",
"f1 IS TRUE",
"true")
- testSqlApi(
+ testAllApis(
+ 'f21.isTrue,
+ "f21.isTrue",
"f21 IS TRUE",
"false")
- testSqlApi(
+ testAllApis(
+ false.isFalse,
+ "false.isFalse",
"FALSE IS FALSE",
"true")
- testSqlApi(
+ testAllApis(
+ 'f21.isFalse,
+ "f21.isFalse",
"f21 IS FALSE",
"false")
- testSqlApi(
+ testAllApis(
+ 'f1.isNotTrue,
+ "f1.isNotTrue",
"f1 IS NOT TRUE",
"false")
- testSqlApi(
+ testAllApis(
+ 'f21.isNotTrue,
+ "f21.isNotTrue",
"f21 IS NOT TRUE",
"true")
- testSqlApi(
+ testAllApis(
+ false.isNotFalse,
+ "false.isNotFalse",
"FALSE IS NOT FALSE",
"false")
- testSqlApi(
+ testAllApis(
+ 'f21.isNotFalse,
+ "f21.isNotFalse",
"f21 IS NOT FALSE",
"true")
}
@@ -2750,8 +3775,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
functions.foreach ( x => {
val tableApiString = x._1.format(field)
val sqlApiString = x._2.format(field)
- testSqlApi(
- // TODO: ignore Table API currently
+ testAllApis(
+ ExpressionParser.parseExpression(tableApiString),
+ tableApiString,
sqlApiString,
"null"
)
@@ -2772,11 +3798,12 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
|.upperCase()
""".stripMargin.format(x)
val sqlApiString = "UPPER(%s)"
- .format("SUBSTRING(%s, 1, -1)")
- .format(x)
+ .format("SUBSTRING(%s, 1, -1)")
+ .format(x)
- testSqlApi(
- // TODO: ignore Table API currently
+ testAllApis(
+ ExpressionParser.parseExpression(tableApiString),
+ tableApiString,
sqlApiString,
"null"
)
@@ -2785,10 +3812,11 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
@Test
def testNullBigDecimal(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f41.sign(),
+ "f41.sign()",
"SIGN(f41)",
"null")
- testSqlApi("SIGN(f41)", "null")
}
@Test
@@ -3177,5 +4205,4 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
"BITNOT(-3)",
"2")
}
-
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
index db80979..b2d72b1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
@@ -20,7 +20,11 @@ package org.apache.flink.table.planner.expressions
import org.apache.flink.api.common.typeinfo.Types
import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.TimeIntervalUnit
import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil
import org.apache.flink.table.planner.utils.DateTimeTestUtil._
import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
import org.apache.flink.types.Row
@@ -36,131 +40,249 @@ class TemporalTypesTest extends ExpressionTestBase {
@Test
def testTimePointLiterals(): Unit = {
- testSqlApi(
+ testAllApis(
+ "1990-10-14".toDate,
+ "'1990-10-14'.toDate",
"DATE '1990-10-14'",
"1990-10-14")
- testSqlApi(
+ testTableApi(
+ localDate2Literal(localDate("2040-09-11")),
+ "'2040-09-11'.toDate",
+ "2040-09-11")
+
+ testAllApis(
+ "1500-04-30".cast(DataTypes.DATE),
+ "'1500-04-30'.cast(SQL_DATE)",
"CAST('1500-04-30' AS DATE)",
"1500-04-30")
- testSqlApi(
+ testAllApis(
+ "15:45:59".toTime,
+ "'15:45:59'.toTime",
"TIME '15:45:59'",
"15:45:59")
- testSqlApi(
+ testTableApi(
+ localTime2Literal(DateTimeTestUtil.localTime("00:00:00")),
+ "'00:00:00'.toTime",
+ "00:00:00")
+
+ testAllApis(
+ "1:30:00".cast(DataTypes.TIME),
+ "'1:30:00'.cast(SQL_TIME)",
"CAST('1:30:00' AS TIME)",
"01:30:00")
- testSqlApi(
+ testAllApis(
+ "1990-10-14 23:00:00.123".toTimestamp,
+ "'1990-10-14 23:00:00.123'.toTimestamp",
"TIMESTAMP '1990-10-14 23:00:00.123'",
"1990-10-14 23:00:00.123")
- testSqlApi(
+ testTableApi(
+ localDateTime2Literal(localDateTime("2040-09-11 00:00:00.000")),
+ "'2040-09-11 00:00:00.000'.toTimestamp",
+ "2040-09-11 00:00:00.000")
+
+ testAllApis(
+ "1500-04-30 12:00:00".cast(DataTypes.TIMESTAMP(3)),
+ "'1500-04-30 12:00:00'.cast(SQL_TIMESTAMP)",
"CAST('1500-04-30 12:00:00' AS TIMESTAMP)",
"1500-04-30 12:00:00.000")
}
@Test
def testTimeIntervalLiterals(): Unit = {
- testSqlApi(
+ testAllApis(
+ 1.year,
+ "1.year",
"INTERVAL '1' YEAR",
"+1-00")
- testSqlApi(
+ testAllApis(
+ 1.month,
+ "1.month",
"INTERVAL '1' MONTH",
"+0-01")
- testSqlApi(
+ testAllApis(
+ 12.days,
+ "12.days",
"INTERVAL '12' DAY",
"+12 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 1.hour,
+ "1.hour",
"INTERVAL '1' HOUR",
"+0 01:00:00.000")
- testSqlApi(
+ testAllApis(
+ 3.minutes,
+ "3.minutes",
"INTERVAL '3' MINUTE",
"+0 00:03:00.000")
- testSqlApi(
+ testAllApis(
+ 3.seconds,
+ "3.seconds",
"INTERVAL '3' SECOND",
"+0 00:00:03.000")
- testSqlApi(
+ testAllApis(
+ 3.millis,
+ "3.millis",
"INTERVAL '0.003' SECOND",
"+0 00:00:00.003")
}
@Test
def testTimePointInput(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0,
+ "f0",
"f0",
"1990-10-14")
- testSqlApi(
+ testAllApis(
+ 'f1,
+ "f1",
"f1",
"10:20:45")
- testSqlApi(
+ testAllApis(
+ 'f2,
+ "f2",
"f2",
"1990-10-14 10:20:45.123")
}
@Test
def testTimeIntervalInput(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f9,
+ "f9",
"f9",
"+2-00")
- testSqlApi(
+ testAllApis(
+ 'f10,
+ "f10",
"f10",
"+0 00:00:12.000")
}
@Test
def testTimePointCasting(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0.cast(DataTypes.TIMESTAMP(3)),
+ "f0.cast(SQL_TIMESTAMP)",
"CAST(f0 AS TIMESTAMP)",
"1990-10-14 00:00:00.000")
- testSqlApi(
+ testAllApis(
+ 'f1.cast(DataTypes.TIMESTAMP(3)),
+ "f1.cast(SQL_TIMESTAMP)",
"CAST(f1 AS TIMESTAMP)",
"1970-01-01 10:20:45.000")
- testSqlApi(
+ testAllApis(
+ 'f2.cast(DataTypes.DATE),
+ "f2.cast(SQL_DATE)",
"CAST(f2 AS DATE)",
"1990-10-14")
- testSqlApi(
+ testAllApis(
+ 'f2.cast(DataTypes.TIME),
+ "f2.cast(SQL_TIME)",
"CAST(f2 AS TIME)",
"10:20:45")
- testSqlApi(
+ testAllApis(
+ 'f2.cast(DataTypes.TIME),
+ "f2.cast(SQL_TIME)",
"CAST(f2 AS TIME)",
"10:20:45")
+ testTableApi(
+ 'f7.cast(DataTypes.DATE),
+ "f7.cast(SQL_DATE)",
+ "2002-11-09")
+
+ testTableApi(
+ 'f7.cast(DataTypes.DATE).cast(DataTypes.INT),
+ "f7.cast(SQL_DATE).cast(INT)",
+ "12000")
+
+ testTableApi(
+ 'f7.cast(DataTypes.TIME),
+ "f7.cast(SQL_TIME)",
+ "00:00:12")
+
+ testTableApi(
+ 'f7.cast(DataTypes.TIME).cast(DataTypes.INT),
+ "f7.cast(SQL_TIME).cast(INT)",
+ "12000")
+
+ testTableApi(
+ 'f15.cast(DataTypes.TIMESTAMP(3)),
+ "f15.cast(SQL_TIMESTAMP)",
+ "2016-06-27 07:23:33.000")
+
+ testTableApi(
+ 'f15.toTimestamp,
+ "f15.toTimestamp",
+ "2016-06-27 07:23:33.000")
+
+ testTableApi(
+ 'f8.cast(DataTypes.TIMESTAMP(3)).cast(DataTypes.BIGINT()),
+ "f8.cast(SQL_TIMESTAMP).cast(LONG)",
+ "1467012213000")
+ }
+
+ @Test
+ def testTimeIntervalCasting(): Unit = {
+ testTableApi(
+ 'f7.cast(DataTypes.INTERVAL(DataTypes.MONTH)),
+ "f7.cast(INTERVAL_MONTHS)",
+ "+1000-00")
+
+ testTableApi(
+ 'f8.cast(DataTypes.INTERVAL(DataTypes.MINUTE())),
+ "f8.cast(INTERVAL_MILLIS)",
+ "+16979 07:23:33.000")
}
@Test
def testTimePointComparison(): Unit = {
- testSqlApi(
+ testAllApis(
+ 'f0 < 'f3,
+ "f0 < f3",
"f0 < f3",
"false")
- testSqlApi(
+ testAllApis(
+ 'f0 < 'f4,
+ "f0 < f4",
"f0 < f4",
"true")
- testSqlApi(
+ testAllApis(
+ 'f1 < 'f5,
+ "f1 < f5",
"f1 < f5",
"false")
- testSqlApi(
+ testAllApis(
+ 'f0.cast(DataTypes.TIMESTAMP(3)) !== 'f2,
+ "f0.cast(SQL_TIMESTAMP) !== f2",
"CAST(f0 AS TIMESTAMP) <> f2",
"true")
- testSqlApi(
+ testAllApis(
+ 'f0.cast(DataTypes.TIMESTAMP(3)) === 'f6,
+ "f0.cast(SQL_TIMESTAMP) === f6",
"CAST(f0 AS TIMESTAMP) = f6",
"true")
}
@@ -170,204 +292,282 @@ class TemporalTypesTest extends ExpressionTestBase {
// interval months comparison
- testSqlApi(
+ testAllApis(
+ 12.months < 24.months,
+ "12.months < 24.months",
"INTERVAL '12' MONTH < INTERVAL '24' MONTH",
"true")
- testSqlApi(
+ testAllApis(
+ 8.years === 8.years,
+ "8.years === 8.years",
"INTERVAL '8' YEAR = INTERVAL '8' YEAR",
"true")
// interval millis comparison
- testSqlApi(
+ testAllApis(
+ 8.millis > 10.millis,
+ "8.millis > 10.millis",
"INTERVAL '0.008' SECOND > INTERVAL '0.010' SECOND",
"false")
- testSqlApi(
+ testAllApis(
+ 8.millis === 8.millis,
+ "8.millis === 8.millis",
"INTERVAL '0.008' SECOND = INTERVAL '0.008' SECOND",
"true")
// interval months addition/subtraction
- testSqlApi(
+ testAllApis(
+ 8.years + 10.months,
+ "8.years + 10.months",
"INTERVAL '8' YEAR + INTERVAL '10' MONTH",
"+8-10")
- testSqlApi(
+ testAllApis(
+ 2.years - 12.months,
+ "2.years - 12.months",
"INTERVAL '2' YEAR - INTERVAL '12' MONTH",
"+1-00")
- testSqlApi(
+ testAllApis(
+ -2.years,
+ "-2.years",
"-INTERVAL '2' YEAR",
"-2-00")
// interval millis addition/subtraction
- testSqlApi(
+ testAllApis(
+ 8.hours + 10.minutes + 12.seconds + 5.millis,
+ "8.hours + 10.minutes + 12.seconds + 5.millis",
"INTERVAL '8' HOUR + INTERVAL '10' MINUTE + INTERVAL '12.005' SECOND",
"+0 08:10:12.005")
- testSqlApi(
+ testAllApis(
+ 1.minute - 10.seconds,
+ "1.minute - 10.seconds",
"INTERVAL '1' MINUTE - INTERVAL '10' SECOND",
"+0 00:00:50.000")
- testSqlApi(
+ testAllApis(
+ -10.seconds,
+ "-10.seconds",
"-INTERVAL '10' SECOND",
"-0 00:00:10.000")
// addition to date
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f0 + 2.days,
+ "f0 + 2.days",
"f0 + INTERVAL '2' DAY",
"1990-10-16")
// interval millis
- testSqlApi(
+ testAllApis(
+ 30.days + 'f0,
+ "30.days + f0",
"INTERVAL '30' DAY + f0",
"1990-11-13")
// interval months
- testSqlApi(
+ testAllApis(
+ 'f0 + 2.months,
+ "f0 + 2.months",
"f0 + INTERVAL '2' MONTH",
"1990-12-14")
// interval months
- testSqlApi(
+ testAllApis(
+ 2.months + 'f0,
+ "2.months + f0",
"INTERVAL '2' MONTH + f0",
"1990-12-14")
// addition to time
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f1 + 12.hours,
+ "f1 + 12.hours",
"f1 + INTERVAL '12' HOUR",
"22:20:45")
// interval millis
- testSqlApi(
+ testAllApis(
+ 12.hours + 'f1,
+ "12.hours + f1",
"INTERVAL '12' HOUR + f1",
"22:20:45")
// addition to timestamp
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f2 + 10.days + 4.millis,
+ "f2 + 10.days + 4.millis",
"f2 + INTERVAL '10 00:00:00.004' DAY TO SECOND",
"1990-10-24 10:20:45.127")
// interval millis
- testSqlApi(
+ testAllApis(
+ 10.days + 'f2 + 4.millis,
+ "10.days + f2 + 4.millis",
"INTERVAL '10 00:00:00.004' DAY TO SECOND + f2",
"1990-10-24 10:20:45.127")
// interval months
- testSqlApi(
+ testAllApis(
+ 'f2 + 10.years,
+ "f2 + 10.years",
"f2 + INTERVAL '10' YEAR",
"2000-10-14 10:20:45.123")
// interval months
- testSqlApi(
+ testAllApis(
+ 10.years + 'f2,
+ "10.years + f2",
"INTERVAL '10' YEAR + f2",
"2000-10-14 10:20:45.123")
// subtraction from date
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f0 - 2.days,
+ "f0 - 2.days",
"f0 - INTERVAL '2' DAY",
"1990-10-12")
// interval millis
- testSqlApi(
+ testAllApis(
+ -30.days + 'f0,
+ "-30.days + f0",
"INTERVAL '-30' DAY + f0",
"1990-09-14")
// interval months
- testSqlApi(
+ testAllApis(
+ 'f0 - 2.months,
+ "f0 - 2.months",
"f0 - INTERVAL '2' MONTH",
"1990-08-14")
// interval months
- testSqlApi(
+ testAllApis(
+ -2.months + 'f0,
+ "-2.months + f0",
"-INTERVAL '2' MONTH + f0",
"1990-08-14")
// subtraction from time
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f1 - 12.hours,
+ "f1 - 12.hours",
"f1 - INTERVAL '12' HOUR",
"22:20:45")
// interval millis
- testSqlApi(
+ testAllApis(
+ -12.hours + 'f1,
+ "-12.hours + f1",
"INTERVAL '-12' HOUR + f1",
"22:20:45")
// subtraction from timestamp
// interval millis
- testSqlApi(
+ testAllApis(
+ 'f2 - 10.days - 4.millis,
+ "f2 - 10.days - 4.millis",
"f2 - INTERVAL '10 00:00:00.004' DAY TO SECOND",
"1990-10-04 10:20:45.119")
// interval millis
- testSqlApi(
+ testAllApis(
+ -10.days + 'f2 - 4.millis,
+ "-10.days + f2 - 4.millis",
"INTERVAL '-10 00:00:00.004' DAY TO SECOND + f2",
"1990-10-04 10:20:45.119")
// interval months
- testSqlApi(
+ testAllApis(
+ 'f2 - 10.years,
+ "f2 - 10.years",
"f2 - INTERVAL '10' YEAR",
"1980-10-14 10:20:45.123")
// interval months
- testSqlApi(
+ testAllApis(
+ -10.years + 'f2,
+ "-10.years + f2",
"INTERVAL '-10' YEAR + f2",
"1980-10-14 10:20:45.123")
// casting
- testSqlApi(
+ testAllApis(
+ -'f9.cast(DataTypes.INTERVAL(DataTypes.MONTH)),
+ "-f9.cast(INTERVAL_MONTHS)",
"-CAST(f9 AS INTERVAL YEAR)",
"-2-00")
- testSqlApi(
+ testAllApis(
+ -'f10.cast(DataTypes.INTERVAL(DataTypes.MINUTE())),
+ "-f10.cast(INTERVAL_MILLIS)",
"-CAST(f10 AS INTERVAL SECOND)",
"-0 00:00:12.000")
// addition/subtraction of interval millis and interval months
- testSqlApi(
+ testAllApis(
+ 'f0 + 2.days + 1.month,
+ "f0 + 2.days + 1.month",
"f0 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
"1990-11-16")
- testSqlApi(
+ testAllApis(
+ 'f0 - 2.days - 1.month,
+ "f0 - 2.days - 1.month",
"f0 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
"1990-09-12")
- testSqlApi(
+ testAllApis(
+ 'f2 + 2.days + 1.month,
+ "f2 + 2.days + 1.month",
"f2 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
"1990-11-16 10:20:45.123")
- testSqlApi(
+ testAllApis(
+ 'f2 - 2.days - 1.month,
+ "f2 - 2.days - 1.month",
"f2 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
"1990-09-12 10:20:45.123")
}
@Test
def testSelectNullValues(): Unit ={
- testSqlApi(
+ testAllApis(
+ 'f11,
+ "f11",
"f11",
"null"
)
- testSqlApi(
+ testAllApis(
+ 'f12,
+ "f12",
"f12",
"null"
)
- testSqlApi(
+ testAllApis(
+ 'f13,
+ "f13",
"f13",
"null"
)
@@ -375,12 +575,16 @@ class TemporalTypesTest extends ExpressionTestBase {
@Test
def testTemporalNullValues() = {
- testSqlApi(
+ testAllApis(
+ 'f13.extract(TimeIntervalUnit.HOUR),
+ "f13.extract(HOUR)",
"extract(HOUR FROM f13)",
"null"
)
- testSqlApi(
+ testAllApis(
+ 'f13.floor(TimeIntervalUnit.HOUR),
+ "f13.floor(HOUR)",
"FLOOR(f13 TO HOUR)",
"null"
)
@@ -402,6 +606,14 @@ class TemporalTypesTest extends ExpressionTestBase {
}
@Test
+ def testdebug() = {
+ testSqlApi("DATE_FORMAT('2018-03-14 01:02:03', 'yyyy/MM/dd HH:mm:ss')",
+ "2018/03/14 01:02:03")
+ testSqlApi("DATE_FORMAT('2018-03-14 01:02:03', 'yyyy-MM-dd HH:mm:ss', " +
+ "'yyyy/MM/dd HH:mm:ss')", "2018/03/14 01:02:03")
+ }
+
+ @Test
def testDateAndTime(): Unit = {
testSqlApi(
"DATE '2018-03-14'",
@@ -675,8 +887,7 @@ class TemporalTypesTest extends ExpressionTestBase {
cases.foreach {
caseExpr =>
- testSqlApi(
- s"CASE WHEN ($caseExpr) is null THEN '$nullable' ELSE '$notNullable' END", nullable)
+ testSqlNullable(caseExpr)
}
}
@@ -702,11 +913,11 @@ class TemporalTypesTest extends ExpressionTestBase {
override def testData: Row = {
val testData = new Row(21)
testData.setField(0, localDate("1990-10-14"))
- testData.setField(1, localTime("10:20:45"))
+ testData.setField(1, DateTimeTestUtil.localTime("10:20:45"))
testData.setField(2, localDateTime("1990-10-14 10:20:45.123"))
testData.setField(3, localDate("1990-10-13"))
testData.setField(4, localDate("1990-10-15"))
- testData.setField(5, localTime("00:00:00"))
+ testData.setField(5, DateTimeTestUtil.localTime("00:00:00"))
testData.setField(6, localDateTime("1990-10-14 00:00:00.0"))
testData.setField(7, 12000)
testData.setField(8, 1467012213000L)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala
new file mode 100644
index 0000000..01585ef
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala
@@ -0,0 +1,485 @@
+/*
+ * 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.{BasicArrayTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{DataTypes, Types, ValidationException}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.planner.expressions.utils.{ExpressionTestBase, _}
+import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions._
+import org.apache.flink.table.planner.utils.DateTimeTestUtil
+import org.apache.flink.types.Row
+
+import org.junit.Test
+
+import java.lang.{Boolean => JBoolean}
+
+class UserDefinedScalarFunctionTest extends ExpressionTestBase {
+
+ @Test
+ def testParameters(): Unit = {
+ testAllApis(
+ Func0('f0),
+ "Func0(f0)",
+ "Func0(f0)",
+ "42")
+
+ testAllApis(
+ Func1('f0),
+ "Func1(f0)",
+ "Func1(f0)",
+ "43")
+
+ testAllApis(
+ Func1('f11),
+ "Func1(f11)",
+ "Func1(f11)",
+ "4")
+
+ testAllApis(
+ Func1('f12),
+ "Func1(f12)",
+ "Func1(f12)",
+ "4")
+
+ testAllApis(
+ Func1('f13),
+ "Func1(f13)",
+ "Func1(f13)",
+ "4.0")
+
+ testAllApis(
+ Func2('f0, 'f1, 'f3),
+ "Func2(f0, f1, f3)",
+ "Func2(f0, f1, f3)",
+ "42 and Test and SimplePojo(Bob,36)")
+
+ testAllApis(
+ Func0(123),
+ "Func0(123)",
+ "Func0(123)",
+ "123")
+
+ // TODO: GenericType with Date/Time/Timestamp -> String would call toString implicitl
+ testAllApis(
+ Func6('f4, 'f5, 'f6),
+ "Func6(f4, f5, f6)",
+ "Func6(f4, f5, f6)",
+ "(1990-10-14,12:10:10,1990-10-14 12:10:10.0)")
+
+ // function names containing keywords
+ testAllApis(
+ Func0('f0),
+ "getFunc0(f0)",
+ "getFunc0(f0)",
+ "42")
+
+ testAllApis(
+ Func0('f0),
+ "asAlways(f0)",
+ "asAlways(f0)",
+ "42")
+
+ testAllApis(
+ Func0('f0),
+ "toWhatever(f0)",
+ "toWhatever(f0)",
+ "42")
+
+ testAllApis(
+ Func0('f0),
+ "Nullable(f0)",
+ "Nullable(f0)",
+ "42")
+
+ // test row type input
+ testAllApis(
+ Func20('f14),
+ "Func20(f14)",
+ "Func20(f14)",
+ "(12,true,(1,2,3))"
+ )
+ }
+
+ @Test
+ def testNullableParameters(): Unit = {
+ testAllApis(
+ Func3(nullOf(DataTypes.INT), nullOf(DataTypes.STRING)),
+ "Func3(Null(INT), Null(STRING))",
+ "Func3(NULL, NULL)",
+ "null and null")
+
+ testAllApis(
+ Func3(nullOf(DataTypes.INT), "Test"),
+ "Func3(Null(INT), 'Test')",
+ "Func3(NULL, 'Test')",
+ "null and Test")
+
+ testAllApis(
+ Func3(42, nullOf(DataTypes.STRING)),
+ "Func3(42, Null(STRING))",
+ "Func3(42, NULL)",
+ "42 and null")
+
+ testAllApis(
+ Func0(nullOf(DataTypes.INT)),
+ "Func0(Null(INT))",
+ "Func0(NULL)",
+ "-1")
+ }
+
+ @Test
+ def testDoubleQuoteParameters(): Unit = {
+ val hello = "\"<hello>\""
+ testAllApis(
+ Func3(42, hello),
+ s"Func3(42, '$hello')",
+ s"Func3(42, '$hello')",
+ s"42 and $hello")
+ }
+
+ @Test
+ def testResults(): Unit = {
+ testAllApis(
+ Func4(),
+ "Func4()",
+ "Func4()",
+ "null")
+
+ testAllApis(
+ Func5(),
+ "Func5()",
+ "Func5()",
+ "-1")
+ }
+
+ @Test
+ def testNesting(): Unit = {
+ testAllApis(
+ Func0(Func0('f0)),
+ "Func0(Func0(f0))",
+ "Func0(Func0(f0))",
+ "42")
+
+ testAllApis(
+ Func0(Func0('f0)),
+ "Func0(Func0(f0))",
+ "Func0(Func0(f0))",
+ "42")
+
+ testAllApis(
+ Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1))),
+ "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
+ "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
+ "8")
+ }
+
+ @Test
+ def testOverloadedParameters(): Unit = {
+ testAllApis(
+ Func8(1),
+ "Func8(1)",
+ "Func8(1)",
+ "a")
+
+ testAllApis(
+ Func8(1, 1),
+ "Func8(1, 1)",
+ "Func8(1, 1)",
+ "b")
+
+ testAllApis(
+ Func8("a", "a"),
+ "Func8('a', 'a')",
+ "Func8('a', 'a')",
+ "c")
+
+ // TODO fix FLINK-13580 to blink-planner
+// testAllApis(
+// Func21('f15),
+// "Func21(f15)",
+// "Func21(f15)",
+// "student#Bob")
+//
+// testAllApis(
+// Func22('f16),
+// "Func22(f16)",
+// "Func22(f16)",
+// "student#Bob")
+ }
+
+ @Test
+ def testTimePointsOnPrimitives(): Unit = {
+ testAllApis(
+ Func9('f4, 'f5, 'f6),
+ "Func9(f4, f5, f6)",
+ "Func9(f4, f5, f6)",
+ "7591 and 43810000 and 655906210000")
+
+ testAllApis(
+ Func10('f6),
+ "Func10(f6)",
+ "Func10(f6)",
+ "1990-10-14 12:10:10.000")
+ }
+
+ @Test
+ def testTimeIntervalsOnPrimitives(): Unit = {
+ testAllApis(
+ Func11('f7, 'f8),
+ "Func11(f7, f8)",
+ "Func11(f7, f8)",
+ "12 and 1000")
+
+ testAllApis(
+ Func12('f8),
+ "Func12(f8)",
+ "Func12(f8)",
+ "+0 00:00:01.000")
+ }
+
+ @Test
+ def testVariableArgs(): Unit = {
+ testAllApis(
+ Func14(1, 2, 3, 4),
+ "Func14(1, 2, 3, 4)",
+ "Func14(1, 2, 3, 4)",
+ "10")
+
+ // Test for empty arguments
+ testAllApis(
+ Func14(),
+ "Func14()",
+ "Func14()",
+ "0")
+
+ // Test for override
+ testAllApis(
+ Func15("Hello"),
+ "Func15('Hello')",
+ "Func15('Hello')",
+ "Hello"
+ )
+
+ testAllApis(
+ Func15('f1),
+ "Func15(f1)",
+ "Func15(f1)",
+ "Test"
+ )
+
+ testAllApis(
+ Func15("Hello", 1, 2, 3),
+ "Func15('Hello', 1, 2, 3)",
+ "Func15('Hello', 1, 2, 3)",
+ "Hello3"
+ )
+
+ testAllApis(
+ Func16('f9),
+ "Func16(f9)",
+ "Func16(f9)",
+ "Hello, World"
+ )
+
+ try {
+ testAllApis(
+ Func17("Hello", "World"),
+ "Func17('Hello', 'World')",
+ "Func17('Hello', 'World')",
+ "Hello, World"
+ )
+ throw new RuntimeException("Shouldn't be reached here!")
+ } catch {
+ case ex: ValidationException =>
+ // ok
+ }
+
+ val JavaFunc2 = new JavaFunc2
+ testAllApis(
+ JavaFunc2("Hi", 1, 3, 5, 7),
+ "JavaFunc2('Hi', 1, 3, 5, 7)",
+ "JavaFunc2('Hi', 1, 3, 5, 7)",
+ "Hi105")
+
+ // test overloading
+ val JavaFunc3 = new JavaFunc3
+ testAllApis(
+ JavaFunc3("Hi"),
+ "JavaFunc3('Hi')",
+ "JavaFunc3('Hi')",
+ "Hi")
+
+ testAllApis(
+ JavaFunc3('f1),
+ "JavaFunc3(f1)",
+ "JavaFunc3(f1)",
+ "Test")
+ }
+
+ @Test
+ def testJavaBoxedPrimitives(): Unit = {
+ val JavaFunc0 = new JavaFunc0()
+ val JavaFunc1 = new JavaFunc1()
+ val JavaFunc4 = new JavaFunc4()
+
+ testAllApis(
+ JavaFunc0('f8),
+ "JavaFunc0(f8)",
+ "JavaFunc0(f8)",
+ "1001"
+ )
+
+ testTableApi(
+ JavaFunc0(1000L),
+ "JavaFunc0(1000L)",
+ "1001"
+ )
+
+ testAllApis(
+ JavaFunc1('f4, 'f5, 'f6),
+ "JavaFunc1(f4, f5, f6)",
+ "JavaFunc1(f4, f5, f6)",
+ "7591 and 43810000 and 655906210000")
+
+ testAllApis(
+ JavaFunc1(nullOf(DataTypes.TIME), 15, nullOf(DataTypes.TIMESTAMP(3))),
+ "JavaFunc1(Null(SQL_TIME), 15, Null(SQL_TIMESTAMP))",
+ "JavaFunc1(NULL, 15, NULL)",
+ "null and 15 and null")
+
+ testAllApis(
+ JavaFunc4('f10, array("a", "b", "c")),
+ "JavaFunc4(f10, array('a', 'b', 'c'))",
+ "JavaFunc4(f10, array['a', 'b', 'c'])",
+ "[1, 2, null] and [a, b, c]"
+ )
+ }
+
+ @Test
+ def testRichFunctions(): Unit = {
+ val richFunc0 = new RichFunc0
+ val richFunc1 = new RichFunc1
+ val richFunc2 = new RichFunc2
+ testAllApis(
+ richFunc0('f0),
+ "RichFunc0(f0)",
+ "RichFunc0(f0)",
+ "43")
+
+ testAllApis(
+ richFunc1('f0),
+ "RichFunc1(f0)",
+ "RichFunc1(f0)",
+ "42")
+
+ testAllApis(
+ richFunc2('f1),
+ "RichFunc2(f1)",
+ "RichFunc2(f1)",
+ "#Test")
+ }
+
+ // ----------------------------------------------------------------------------------------------
+
+ override def testData: Row = {
+ val testData = new Row(17)
+ testData.setField(0, 42)
+ testData.setField(1, "Test")
+ testData.setField(2, null)
+ testData.setField(3, SimplePojo("Bob", 36))
+ testData.setField(4, DateTimeTestUtil.localDate("1990-10-14"))
+ testData.setField(5, DateTimeTestUtil.localTime("12:10:10"))
+ testData.setField(6, DateTimeTestUtil.localDateTime("1990-10-14 12:10:10"))
+ testData.setField(7, 12)
+ testData.setField(8, 1000L)
+ testData.setField(9, Seq("Hello", "World"))
+ testData.setField(10, Array[Integer](1, 2, null))
+ testData.setField(11, 3.toByte)
+ testData.setField(12, 3.toShort)
+ testData.setField(13, 3.toFloat)
+ testData.setField(14, Row.of(
+ 12.asInstanceOf[Integer],
+ true.asInstanceOf[JBoolean],
+ Row.of(1.asInstanceOf[Integer], 2.asInstanceOf[Integer], 3.asInstanceOf[Integer]))
+ )
+ testData.setField(15, new GraduatedStudent("Bob"))
+ testData.setField(16, Array(new GraduatedStudent("Bob")))
+ testData
+ }
+
+ override def typeInfo: RowTypeInfo = {
+ new RowTypeInfo(
+ Types.INT,
+ Types.STRING,
+ Types.BOOLEAN,
+ TypeInformation.of(classOf[SimplePojo]),
+ Types.LOCAL_DATE,
+ Types.LOCAL_TIME,
+ Types.LOCAL_DATE_TIME,
+ Types.INTERVAL_MONTHS,
+ Types.INTERVAL_MILLIS,
+ TypeInformation.of(classOf[Seq[String]]),
+ BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO,
+ Types.BYTE,
+ Types.SHORT,
+ Types.FLOAT,
+ Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)),
+ TypeInformation.of(classOf[GraduatedStudent]),
+ TypeInformation.of(classOf[Array[GraduatedStudent]])
+ )
+ }
+
+ override def functions: Map[String, ScalarFunction] = Map(
+ "Func0" -> Func0,
+ "getFunc0" -> Func0,
+ "asAlways" -> Func0,
+ "toWhatever" -> Func0,
+ "Nullable" -> Func0,
+ "Func1" -> Func1,
+ "Func2" -> Func2,
+ "Func3" -> Func3,
+ "Func4" -> Func4,
+ "Func5" -> Func5,
+ "Func6" -> Func6,
+ "Func7" -> Func7,
+ "Func8" -> Func8,
+ "Func9" -> Func9,
+ "Func10" -> Func10,
+ "Func11" -> Func11,
+ "Func12" -> Func12,
+ "Func14" -> Func14,
+ "Func15" -> Func15,
+ "Func16" -> Func16,
+ "Func17" -> Func17,
+ "Func19" -> Func19,
+ "Func20" -> Func20,
+ "JavaFunc0" -> new JavaFunc0,
+ "JavaFunc1" -> new JavaFunc1,
+ "JavaFunc2" -> new JavaFunc2,
+ "JavaFunc3" -> new JavaFunc3,
+ "JavaFunc4" -> new JavaFunc4,
+ "RichFunc0" -> new RichFunc0,
+ "RichFunc1" -> new RichFunc1,
+ "RichFunc2" -> new RichFunc2
+ )
+}
+
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
index 9b7bd4a..02cc054 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
@@ -26,10 +26,14 @@ import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
import org.apache.flink.table.api.internal.TableEnvironmentImpl
import org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.api.{EnvironmentSettings, TableConfig}
import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, DataFormatConverters}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.functions.ScalarFunction
import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
import org.apache.flink.table.planner.delegation.PlannerBase
+import org.apache.flink.table.planner.expressions.ExpressionBuilder
import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
import org.apache.flink.table.types.DataType
import org.apache.flink.table.types.logical.{RowType, VarCharType}
@@ -77,10 +81,13 @@ abstract class ExpressionTestBase {
@Rule
def thrown: ExpectedException = expectedException
+ def functions: Map[String, ScalarFunction] = Map()
+
@Before
def prepare(): Unit = {
val ds = env.fromCollection(Collections.emptyList[Row](), typeInfo)
tEnv.registerDataStream(tableName, ds)
+ functions.foreach(f => tEnv.registerFunction(f._1, f._2))
// prepare RelBuilder
relBuilder.scan(tableName)
@@ -161,12 +168,13 @@ abstract class ExpressionTestBase {
null
}
+ val original = if (originalExpr == null) "" else s"for: [$originalExpr]"
+
assertEquals(
- s"Wrong result for: [$originalExpr] optimized to: [$optimizedExpr]",
+ s"Wrong result $original optimized to: [$optimizedExpr]",
expected,
if (actual == null) "null" else actual)
}
-
}
private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = {
@@ -174,11 +182,14 @@ abstract class ExpressionTestBase {
val parsed = calcitePlanner.parse(s"SELECT $sqlExpr FROM $tableName")
val validated = calcitePlanner.validate(parsed)
val converted = calcitePlanner.rel(validated).rel
+ addTestExpr(converted, expected, sqlExpr)
+ }
+ private def addTestExpr(relNode: RelNode, expected: String, summaryString: String): Unit = {
val builder = new HepProgramBuilder()
builder.addRuleInstance(ProjectToCalcRule.INSTANCE)
val hep = new HepPlanner(builder.build())
- hep.setRoot(converted)
+ hep.setRoot(relNode)
val optimized = hep.findBestExp()
// throw exception if plan contains more than a calc
@@ -186,7 +197,7 @@ abstract class ExpressionTestBase {
fail("Expression is converted into more than a Calc operation. Use a different test method.")
}
- testExprs += ((sqlExpr, extractRexNode(optimized), expected))
+ testExprs += ((summaryString, extractRexNode(optimized), expected))
}
private def extractRexNode(node: RelNode): RexNode = {
@@ -196,10 +207,62 @@ abstract class ExpressionTestBase {
calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
}
+ def testAllApis(
+ expr: Expression,
+ exprString: String,
+ sqlExpr: String,
+ expected: String): Unit = {
+ addTableApiTestExpr(expr, expected)
+ addTableApiTestExpr(exprString, expected)
+ addSqlTestExpr(sqlExpr, expected)
+ if (expected == nullable) {
+ testTableNullable(expr, exprString)
+ testSqlNullable(sqlExpr)
+ }
+ }
+
+ def testTableApi(
+ expr: Expression,
+ exprString: String,
+ expected: String): Unit = {
+ addTableApiTestExpr(expr, expected)
+ addTableApiTestExpr(exprString, expected)
+ if (expected == nullable) {
+ testTableNullable(expr, exprString)
+ }
+ }
+
+ private def addTableApiTestExpr(tableApiString: String, expected: String): Unit = {
+ addTableApiTestExpr(ExpressionParser.parseExpression(tableApiString), expected)
+ }
+
+ private def addTableApiTestExpr(tableApiExpr: Expression, expected: String): Unit = {
+ // create RelNode from Table API expression
+ val relNode = relBuilder
+ .queryOperation(tEnv.scan(tableName).select(tableApiExpr).getQueryOperation).build()
+ addTestExpr(relNode, expected, tableApiExpr.asSummaryString())
+ }
+
+ def testSqlNullable(nullUdf: String): Unit = {
+ addSqlTestExpr(
+ s"CASE WHEN ($nullUdf) is null THEN '$nullable' ELSE '$notNullable' END", nullable)
+ }
+
def testSqlApi(
sqlExpr: String,
expected: String): Unit = {
addSqlTestExpr(sqlExpr, expected)
+ if (expected == nullable) {
+ testSqlNullable(sqlExpr)
+ }
+ }
+
+ def testTableNullable(nullExpr: Expression, nullExprString: String): Unit = {
+ val retExpr = ExpressionBuilder.ifThenElse(nullExpr.isNull, nullable, notNullable)
+ addTableApiTestExpr(retExpr, nullable)
+ val retStrExpr = ifThenElse(
+ ExpressionParser.parseExpression(nullExprString).isNull, nullable, notNullable)
+ addTableApiTestExpr(retStrExpr, nullable)
}
def testData: Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
index 116c40b..7a3b224 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.expressions.utils
import org.apache.flink.api.common.typeinfo.Types
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.table.dataformat.Decimal
+import org.apache.flink.table.functions.ScalarFunction
import org.apache.flink.table.planner.utils.DateTimeTestUtil._
import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
import org.apache.flink.types.Row
@@ -72,4 +73,8 @@ abstract class ScalarOperatorsTestBase extends ExpressionTestBase {
/* 17 */ DecimalTypeInfo.of(19, 1)
)
}
+
+ override def functions: Map[String, ScalarFunction] = Map(
+ "shouldNotExecuteFunc" -> ShouldNotExecuteFunc
+ )
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
index 485878a..9faed1e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
@@ -342,6 +342,28 @@ object Func20 extends ScalarFunction {
}
@SerialVersionUID(1L)
+object Func21 extends ScalarFunction {
+ def eval(p: People): String = {
+ p.name
+ }
+
+ def eval(p: Student): String = {
+ "student#" + p.name
+ }
+}
+
+@SerialVersionUID(1L)
+object Func22 extends ScalarFunction {
+ def eval(a: Array[People]): String = {
+ a.head.name
+ }
+
+ def eval(a: Array[Student]): String = {
+ "student#" + a.head.name
+ }
+}
+
+@SerialVersionUID(1L)
object Func23 extends ScalarFunction {
def eval(a: Integer, b: JLong, c: String): Row = {
Row.of("star", a, b, c)
@@ -410,3 +432,9 @@ class SplitUDF(deterministic: Boolean) extends ScalarFunction {
override def isDeterministic: Boolean = deterministic
}
+
+class People(val name: String)
+
+class Student(name: String) extends People(name)
+
+class GraduatedStudent(name: String) extends Student(name)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
index 23ec7ba..86dbe45 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.expressions.validation
import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase
import org.junit.Test
@@ -26,16 +27,31 @@ import org.junit.Test
class ArrayTypeValidationTest extends ArrayTypeTestBase {
@Test(expected = classOf[ValidationException])
+ def testImplicitTypeCastTableApi(): Unit = {
+ testTableApi(array(1.0, 2.0f), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
def testImplicitTypeCastArraySql(): Unit = {
testSqlApi("ARRAY['string', 12]", "FAIL")
}
@Test(expected = classOf[ValidationException])
+ def testObviousInvalidIndexTableApi(): Unit = {
+ testTableApi('f2.at(0), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
def testEmptyArraySql(): Unit = {
testSqlApi("ARRAY[]", "FAIL")
}
@Test(expected = classOf[ValidationException])
+ def testEmptyArrayTableApi(): Unit = {
+ testTableApi("FAIL", "array()", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
def testNullArraySql(): Unit = {
testSqlApi("ARRAY[NULL]", "FAIL")
}
@@ -46,6 +62,28 @@ class ArrayTypeValidationTest extends ArrayTypeTestBase {
}
@Test(expected = classOf[ValidationException])
+ def testDifferentTypesArrayTableApi(): Unit = {
+ testTableApi("FAIL", "array(1, true)", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testUnsupportedComparison(): Unit = {
+ testAllApis(
+ 'f2 <= 'f5.at(1),
+ "f2 <= f5.at(1)",
+ "f2 <= f5[1]",
+ "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testElementNonArray(): Unit = {
+ testTableApi(
+ 'f0.element(),
+ "FAIL",
+ "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
def testElementNonArraySql(): Unit = {
testSqlApi(
"ELEMENT(f0)",
@@ -53,6 +91,11 @@ class ArrayTypeValidationTest extends ArrayTypeTestBase {
}
@Test(expected = classOf[ValidationException])
+ def testCardinalityOnNonArray(): Unit = {
+ testTableApi('f0.cardinality(), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
def testCardinalityOnNonArraySql(): Unit = {
testSqlApi("CARDINALITY(f0)", "FAIL")
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
index 3657fce..687700e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.expressions.validation
import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.CompositeTypeTestBase
import org.junit.Test
@@ -34,6 +35,26 @@ class CompositeAccessValidationTest extends CompositeTypeTestBase {
def testWrongSqlField(): Unit = {
testSqlApi("f5.test", "13")
}
+
+ @Test(expected = classOf[ValidationException])
+ def testWrongIntKeyField(): Unit = {
+ testTableApi('f0.get(555), "'fail'", "fail")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testWrongIntKeyField2(): Unit = {
+ testTableApi("fail", "f0.get(555)", "fail")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testWrongStringKeyField(): Unit = {
+ testTableApi('f0.get("fghj"), "'fail'", "fail")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testWrongStringKeyField2(): Unit = {
+ testTableApi("fail", "f0.get('fghj')", "fail")
+ }
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
index 05c01f8..52a62c7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.expressions.validation
import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.expressions.utils.MapTypeTestBase
import org.junit.Test
@@ -27,16 +28,23 @@ class MapTypeValidationTest extends MapTypeTestBase {
@Test(expected = classOf[ValidationException])
def testWrongKeyType(): Unit = {
- testSqlApi("f2[12]", "FAIL")
+ testAllApis('f2.at(12), "f2.at(12)", "f2[12]", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testIncorrectMapTypeComparison(): Unit = {
+ testAllApis('f1 === 'f3, "f1 === f3", "f1 = f3", "FAIL")
}
@Test(expected = classOf[ValidationException])
def testUnsupportedComparisonType(): Unit = {
+ testAllApis('f6 !== 'f2, "f6 !== f2", "f6 != f2", "FAIL")
testSqlApi("f6 <> f2", "FAIL")
}
@Test(expected = classOf[ValidationException])
def testEmptyMap(): Unit = {
+ testAllApis("FAIL", "map()", "MAP[]", "FAIL")
testSqlApi("MAP[]", "FAIL")
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
index f1297be..f72de54 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.expressions.validation
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.api.{SqlParserException, ValidationException}
import org.apache.flink.table.planner.expressions.utils.RowTypeTestBase
@@ -32,11 +33,11 @@ class RowTypeValidationTest extends RowTypeTestBase {
@Test(expected = classOf[ValidationException])
def testNullRowType(): Unit = {
- testSqlApi("Row(NULL)", "FAIL")
+ testAllApis("FAIL", "row(null)", "Row(NULL)", "FAIL")
}
@Test(expected = classOf[ValidationException])
def testSqlRowIllegalAccess(): Unit = {
- testSqlApi("f5.f2", "FAIL")
+ testAllApis('f5.get("f2"), "f5.get('f2')", "f5.f2", "FAIL")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
index 4f791ad..da4763f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
@@ -18,11 +18,13 @@
package org.apache.flink.table.planner.expressions.validation
+import org.apache.flink.table.api.scala._
import org.apache.flink.table.api.{SqlParserException, ValidationException}
+import org.apache.flink.table.expressions.TimePointUnit
import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase
import org.apache.calcite.avatica.util.TimeUnit
-import org.junit.Test
+import org.junit.{Ignore, Test}
class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
@@ -30,45 +32,75 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
// Math functions
// ----------------------------------------------------------------------------------------------
+ @Ignore
@Test
- def testInvalidBin1(): Unit = {
+ def testInvalidLog1(): Unit = {
thrown.expect(classOf[ValidationException])
- testSqlApi("BIN(f12)", "101010") // float type
+ // invalid arithmetic argument
+ testSqlApi(
+ "LOG(1, 100)",
+ "FAIL"
+ )
}
+ @Ignore
@Test
- def testInvalidBin2(): Unit = {
+ def testInvalidLog2(): Unit ={
thrown.expect(classOf[ValidationException])
+ // invalid arithmetic argument
+ testSqlApi(
+ "LOG(-1)",
+ "FAIL"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBin1(): Unit = {
+ testSqlApi("BIN(f12)", "101010") // float type
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBin2(): Unit = {
testSqlApi("BIN(f15)", "101010") // BigDecimal type
}
- @Test
+ @Test(expected = classOf[ValidationException])
def testInvalidBin3(): Unit = {
- thrown.expect(classOf[ValidationException])
testSqlApi("BIN(f16)", "101010") // Date type
}
+ // ----------------------------------------------------------------------------------------------
+ // String functions
+ // ----------------------------------------------------------------------------------------------
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidSubstring1(): Unit = {
+ // Must fail. Parameter of substring must be an Integer not a Double.
+ testTableApi("test".substring(2.0.toExpr), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidSubstring2(): Unit = {
+ // Must fail. Parameter of substring must be an Integer not a String.
+ testTableApi("test".substring("test".toExpr), "FAIL", "FAIL")
+ }
// ----------------------------------------------------------------------------------------------
// Temporal functions
// ----------------------------------------------------------------------------------------------
- @Test
- def testTimestampAddWithWrongTimestampInterval(): Unit = {
- thrown.expect(classOf[SqlParserException])
- testSqlApi("TIMESTAMPADD(XXX, 1, timestamp '2016-02-24')", "2016-06-16")
+ @Test(expected = classOf[SqlParserException])
+ def testTimestampAddWithWrongTimestampInterval(): Unit ={
+ testSqlApi("TIMESTAMPADD(XXX, 1, timestamp '2016-02-24'))", "2016-06-16")
}
- @Test
- def testTimestampAddWithWrongTimestampFormat(): Unit = {
- thrown.expect(classOf[SqlParserException])
- thrown.expectMessage("Illegal TIMESTAMP literal '2016/02/24'")
- testSqlApi("TIMESTAMPADD(YEAR, 1, timestamp '2016/02/24')", "2016-06-16")
+ @Test(expected = classOf[SqlParserException])
+ def testTimestampAddWithWrongTimestampFormat(): Unit ={
+ testSqlApi("TIMESTAMPADD(YEAR, 1, timestamp '2016-02-24'))", "2016-06-16")
}
- @Test
- def testTimestampAddWithWrongQuantity(): Unit = {
- thrown.expect(classOf[ValidationException])
+ @Test(expected = classOf[ValidationException])
+ def testTimestampAddWithWrongQuantity(): Unit ={
testSqlApi("TIMESTAMPADD(YEAR, 1.0, timestamp '2016-02-24 12:42:25')", "2016-06-16")
}
@@ -76,6 +108,50 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
// Sub-query functions
// ----------------------------------------------------------------------------------------------
+ @Test(expected = classOf[ValidationException])
+ def testInValidationExceptionMoreThanOneTypes(): Unit = {
+ testTableApi(
+ 'f2.in('f3, 'f8),
+ "f2.in(f3, f8)",
+ "true"
+ )
+ testTableApi(
+ 'f2.in('f3, 'f4, 4),
+ "f2.in(f3, f4, 4)",
+ "false" // OK if all numeric
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def scalaInValidationExceptionDifferentOperandsTest(): Unit = {
+ testTableApi(
+ 'f1.in("Hi", "Hello world", "Comment#1"),
+ "true",
+ "true"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def javaInValidationExceptionDifferentOperandsTest(): Unit = {
+ testTableApi(
+ true,
+ "f1.in('Hi','Hello world','Comment#1')",
+ "true"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testTimestampDiffWithWrongTime(): Unit = {
+ testTableApi(
+ timestampDiff(TimePointUnit.DAY, "2016-02-24", "2016-02-27"), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testTimestampDiffWithWrongTimeAndUnit(): Unit = {
+ testTableApi(
+ timestampDiff(TimePointUnit.MINUTE, "2016-02-24", "2016-02-27"), "FAIL", "FAIL")
+ }
+
@Test
def testDOWWithTimeWhichIsUnsupported(): Unit = {
thrown.expect(classOf[ValidationException])
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala
new file mode 100644
index 0000000..9bc2ea8
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala
@@ -0,0 +1,88 @@
+/*
+ * 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.validation
+
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.planner.expressions.utils.ScalarOperatorsTestBase
+
+import org.junit.Test
+
+class ScalarOperatorsValidationTest extends ScalarOperatorsTestBase {
+
+ @Test(expected = classOf[ValidationException])
+ def testIfInvalidTypesScala(): Unit = {
+ testTableApi(('f6 && true).?(5, "false"), "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testIfInvalidTypesJava(): Unit = {
+ testTableApi("FAIL", "(f8 && true).?(5, 'false')", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidStringComparison1(): Unit = {
+ testTableApi("w" === 4, "FAIL", "FAIL")
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidStringComparison2(): Unit = {
+ testTableApi("w" > 4.toExpr, "FAIL", "FAIL")
+ }
+
+ // ----------------------------------------------------------------------------------------------
+ // Sub-query functions
+ // ----------------------------------------------------------------------------------------------
+
+ @Test(expected = classOf[ValidationException])
+ def testInMoreThanOneTypes(): Unit = {
+ testTableApi(
+ 'f2.in('f3, 'f4, 4),
+ "FAIL",
+ "FAIL"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInDifferentOperands(): Unit = {
+ testTableApi(
+ 'f1.in("Hi", "Hello world", "Comment#1"),
+ "FAIL",
+ "FAIL"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testBetweenWithDifferentOperandTypeScala(): Unit = {
+ testTableApi(
+ 2.between(1, "a"),
+ "FAIL",
+ "FAIL"
+ )
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testBetweenWithDifferentOperandTypeJava(): Unit = {
+ testTableApi(
+ "FAIL",
+ "2.between(1, 'a')",
+ "FAIL"
+ )
+ }
+}