You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2021/11/02 08:47:17 UTC
[flink] 02/03: [FLINK-24454][table-planner][tests] Consolidate cast
tests
This is an automated email from the ASF dual-hosted git repository.
twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit de487cc6933d31e6659c7ef4589df4445d8cbfa6
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Fri Oct 29 15:49:23 2021 +0200
[FLINK-24454][table-planner][tests] Consolidate cast tests
Check for `CAST` tests in other classes, move them in `CastFunctionITCase`,
or completely remove them if are already contained in `CastFunctionITCase`.
This closes #17579.
---
.../planner/functions/CastFunctionITCase.java | 977 ++++++++++++++-------
.../planner/expressions/DecimalCastTest.scala | 330 -------
.../planner/expressions/DecimalTypeTest.scala | 131 ---
.../planner/expressions/ScalarOperatorsTest.scala | 8 -
.../planner/runtime/batch/sql/CalcITCase.scala | 2 +-
.../planner/runtime/batch/table/CalcITCase.scala | 32 -
.../planner/runtime/stream/sql/CalcITCase.scala | 21 -
7 files changed, 645 insertions(+), 856 deletions(-)
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
index a365eef..723a262 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
@@ -23,6 +23,9 @@ import org.apache.flink.table.api.config.TableConfigOptions;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.types.AbstractDataType;
import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.junit.runners.Parameterized;
@@ -37,6 +40,7 @@ import java.time.ZoneId;
import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
import static org.apache.flink.table.api.DataTypes.ARRAY;
@@ -53,6 +57,7 @@ import static org.apache.flink.table.api.DataTypes.FLOAT;
import static org.apache.flink.table.api.DataTypes.INT;
import static org.apache.flink.table.api.DataTypes.INTERVAL;
import static org.apache.flink.table.api.DataTypes.MONTH;
+import static org.apache.flink.table.api.DataTypes.SECOND;
import static org.apache.flink.table.api.DataTypes.SMALLINT;
import static org.apache.flink.table.api.DataTypes.STRING;
import static org.apache.flink.table.api.DataTypes.TIME;
@@ -70,6 +75,34 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
private static final ZoneId TEST_TZ = ZoneId.of("Asia/Shanghai");
private static final ZoneOffset TEST_OFFSET = ZoneOffset.ofHoursMinutes(-1, -20);
+ private static final byte[] DEFAULT_BINARY = new byte[] {0, 1};
+ private static final byte[] DEFAULT_VARBINARY = new byte[] {0, 1, 2};
+ private static final byte[] DEFAULT_BYTES = new byte[] {0, 1, 2, 3, 4};
+
+ private static final byte DEFAULT_POSITIVE_TINY_INT = (byte) 5;
+ private static final byte DEFAULT_NEGATIVE_TINY_INT = (byte) -5;
+ private static final short DEFAULT_POSITIVE_SMALL_INT = (short) 12345;
+ private static final short DEFAULT_NEGATIVE_SMALL_INT = (short) -12345;
+ private static final int DEFAULT_POSITIVE_INT = 1234567;
+ private static final int DEFAULT_NEGATIVE_INT = -1234567;
+ private static final long DEFAULT_POSITIVE_BIGINT = 12345678901L;
+ private static final long DEFAULT_NEGATIVE_BIGINT = -12345678901L;
+ private static final float DEFAULT_POSITIVE_FLOAT = 123.456f;
+ private static final float DEFAULT_NEGATIVE_FLOAT = -123.456f;
+ private static final double DEFAULT_POSITIVE_DOUBLE = 123.456789d;
+ private static final double DEFAULT_NEGATIVE_DOUBLE = -123.456789d;
+
+ private static final LocalDate DEFAULT_DATE = LocalDate.parse("2021-09-24");
+ private static final LocalTime DEFAULT_TIME = LocalTime.parse("12:34:56.123");
+ private static final LocalDateTime DEFAULT_TIMESTAMP =
+ LocalDateTime.parse("2021-09-24T12:34:56.1234567");
+ private static final Instant DEFAULT_TIMESTAMP_LTZ = fromLocalTZ("2021-09-24T22:34:56.1234567");
+
+ private static final Period DEFAULT_INTERVAL_YEAR = Period.of(10, 4, 0);
+ private static final Duration DEFAULT_INTERVAL_DAY = Duration.ofHours(12);
+
+ private static final int[] DEFAULT_ARRAY = new int[] {0, 1, 2};
+
@Override
protected Configuration configuration() {
return super.configuration().set(TableConfigOptions.LOCAL_TIME_ZONE, TEST_TZ.getId());
@@ -77,8 +110,17 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
@Parameterized.Parameters(name = "{index}: {0}")
public static List<TestSpec> testData() {
+ final List<TestSpec> specs = new ArrayList<>();
+ specs.addAll(allTypesBasic());
+ specs.addAll(decimalCasts());
+ specs.addAll(numericBounds());
+ return specs;
+ }
+
+ public static List<TestSpec> allTypesBasic() {
return Arrays.asList(
CastTestSpecBuilder.testCastTo(CHAR(3))
+ .fromCase(CHAR(5), null, null)
.fromCase(CHAR(3), "foo", "foo")
.fromCase(CHAR(4), "foo", "foo ")
.fromCase(CHAR(4), "foo ", "foo ")
@@ -88,9 +130,10 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// https://issues.apache.org/jira/browse/FLINK-24413 - Trim to precision
// in this case down to 3 chars
.fromCase(STRING(), "abcdef", "abcdef") // "abc"
- .fromCase(DATE(), LocalDate.parse("2021-09-30"), "2021-09-30") // "202"
+ .fromCase(DATE(), DEFAULT_DATE, "2021-09-24") // "202"
.build(),
CastTestSpecBuilder.testCastTo(VARCHAR(3))
+ .fromCase(VARCHAR(5), null, null)
.fromCase(CHAR(3), "foo", "foo")
.fromCase(CHAR(4), "foo", "foo ")
.fromCase(CHAR(4), "foo ", "foo ")
@@ -110,36 +153,65 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(STRING(), "Apache Flink", "Apache Flink")
.fromCase(STRING(), null, null)
.fromCase(BOOLEAN(), true, "true")
- .fromCase(BINARY(2), new byte[] {0, 1}, "\u0000\u0001")
- .fromCase(BINARY(3), new byte[] {0, 1}, "\u0000\u0001\u0000")
- .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, "\u0000\u0001\u0002")
- .fromCase(VARBINARY(5), new byte[] {0, 1, 2}, "\u0000\u0001\u0002")
- .fromCase(
- BYTES(),
- new byte[] {0, 1, 2, 3, 4},
- "\u0000\u0001\u0002\u0003\u0004")
+ .fromCase(BINARY(2), DEFAULT_BINARY, "\u0000\u0001")
+ .fromCase(BINARY(3), DEFAULT_BINARY, "\u0000\u0001\u0000")
+ .fromCase(VARBINARY(3), DEFAULT_VARBINARY, "\u0000\u0001\u0002")
+ .fromCase(VARBINARY(5), DEFAULT_VARBINARY, "\u0000\u0001\u0002")
+ .fromCase(BYTES(), DEFAULT_BYTES, "\u0000\u0001\u0002\u0003\u0004")
.fromCase(DECIMAL(4, 3), 9.87, "9.870")
+ .fromCase(DECIMAL(10, 5), 1, "1.00000")
// https://issues.apache.org/jira/browse/FLINK-24403 - Left zero padding
// currently not working
// .fromCase(DECIMAL(5, 3), 09.87, "09.870")
- .fromCase(TINYINT(), -125, "-125")
- .fromCase(SMALLINT(), 32767, "32767")
- .fromCase(INT(), -12345678, "-12345678")
- .fromCase(BIGINT(), 1234567891234L, "1234567891234")
- .fromCase(FLOAT(), -123.456, "-123.456")
- .fromCase(DOUBLE(), 12345.678901, "12345.678901")
- .fromCase(DATE(), LocalDate.parse("2021-09-24"), "2021-09-24")
- // https://issues.apache.org/jira/browse/FLINK-17224 Currently, fractional
- // seconds are lost
- .fromCase(TIME(5), LocalTime.parse("12:34:56.1234567"), "12:34:56")
.fromCase(
- TIMESTAMP(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- "2021-09-24 12:34:56.123456")
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ String.valueOf(DEFAULT_POSITIVE_TINY_INT))
.fromCase(
- TIMESTAMP(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- "2021-09-24 12:34:56.1234")
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ String.valueOf(DEFAULT_NEGATIVE_TINY_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_POSITIVE_SMALL_INT,
+ String.valueOf(DEFAULT_POSITIVE_SMALL_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_NEGATIVE_SMALL_INT,
+ String.valueOf(DEFAULT_NEGATIVE_SMALL_INT))
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, String.valueOf(DEFAULT_POSITIVE_INT))
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, String.valueOf(DEFAULT_NEGATIVE_INT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_POSITIVE_BIGINT,
+ String.valueOf(DEFAULT_POSITIVE_BIGINT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_NEGATIVE_BIGINT,
+ String.valueOf(DEFAULT_NEGATIVE_BIGINT))
+ .fromCase(
+ FLOAT(),
+ DEFAULT_POSITIVE_FLOAT,
+ String.valueOf(DEFAULT_POSITIVE_FLOAT))
+ .fromCase(
+ FLOAT(),
+ DEFAULT_NEGATIVE_FLOAT,
+ String.valueOf(DEFAULT_NEGATIVE_FLOAT))
+ .fromCase(
+ DOUBLE(),
+ DEFAULT_POSITIVE_DOUBLE,
+ String.valueOf(DEFAULT_POSITIVE_DOUBLE))
+ .fromCase(
+ DOUBLE(),
+ DEFAULT_NEGATIVE_DOUBLE,
+ String.valueOf(DEFAULT_NEGATIVE_DOUBLE))
+ .fromCase(DATE(), DEFAULT_DATE, "2021-09-24")
+ // https://issues.apache.org/jira/browse/FLINK-17224 Currently, fractional
+ // seconds are lost
+ .fromCase(TIME(5), DEFAULT_TIME, "12:34:56")
+ .fromCase(TIMESTAMP(), DEFAULT_TIMESTAMP, "2021-09-24 12:34:56.123456")
+ .fromCase(TIMESTAMP(8), DEFAULT_TIMESTAMP, "2021-09-24 12:34:56.12345670")
+ .fromCase(TIMESTAMP(4), DEFAULT_TIMESTAMP, "2021-09-24 12:34:56.1234")
.fromCase(TIMESTAMP(4).nullable(), null, null)
// https://issues.apache.org/jira/browse/FLINK-20869
@@ -147,8 +219,8 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(
TIMESTAMP_LTZ(5),
- fromLocalTZ("2021-09-24T18:34:56.123456"),
- "2021-09-25 03:54:56.12345")
+ DEFAULT_TIMESTAMP_LTZ,
+ "2021-09-25 07:54:56.12345")
.fromCase(INTERVAL(YEAR()), 84, "+7-00")
.fromCase(INTERVAL(MONTH()), 5, "+0-05")
.fromCase(INTERVAL(MONTH()), 123, "+10-03")
@@ -157,13 +229,14 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(INTERVAL(DAY()), 123456789L, "+1 10:17:36.789")
.fromCase(INTERVAL(DAY()), Duration.ofHours(36), "+1 12:00:00.000")
// https://issues.apache.org/jira/browse/FLINK-21456 Not supported currently
- // ARRAY
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(BOOLEAN())
+ .fromCase(BOOLEAN(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(CHAR(4), "true", true)
.fromCase(VARCHAR(5), "FalsE", false)
@@ -173,65 +246,72 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, true)
.fromCase(BOOLEAN(), false, false)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
// https://issues.apache.org/jira/browse/FLINK-24576 should also fail for
// SQL
.failTableApi(DECIMAL(4, 3), 4.3)
- .fromCase(TINYINT(), -125, true)
+ .fromCase(TINYINT(), DEFAULT_POSITIVE_TINY_INT, true)
+ .fromCase(TINYINT(), DEFAULT_NEGATIVE_TINY_INT, true)
.fromCase(TINYINT(), 0, false)
- .fromCase(SMALLINT(), 32767, true)
+ .fromCase(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT, true)
+ .fromCase(SMALLINT(), DEFAULT_NEGATIVE_SMALL_INT, true)
.fromCase(SMALLINT(), 0, false)
- .fromCase(INT(), -12345678, true)
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, true)
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, true)
.fromCase(INT(), 0, false)
- .fromCase(BIGINT(), 1234567891234L, true)
+ .fromCase(BIGINT(), DEFAULT_POSITIVE_BIGINT, true)
+ .fromCase(BIGINT(), DEFAULT_NEGATIVE_BIGINT, true)
.fromCase(BIGINT(), 0, false)
// https://issues.apache.org/jira/browse/FLINK-24576 should also fail for
// SQL
.failTableApi(FLOAT(), -123.456)
.failTableApi(DOUBLE(), 0)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(BINARY(2))
+ .fromCase(BINARY(5), null, null)
.fromCase(CHAR(3), "foo", new byte[] {102, 111, 111})
.fromCase(VARCHAR(5), "Flink", new byte[] {70, 108, 105, 110, 107})
// https://issues.apache.org/jira/browse/FLINK-24419 - not trimmed to 2
// bytes
.fromCase(STRING(), "Apache", new byte[] {65, 112, 97, 99, 104, 101})
// Not supported - no fix
- // BOOLEAN
- .fromCase(BINARY(2), new byte[] {0, 1}, new byte[] {0, 1})
- .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, new byte[] {0, 1, 2})
- .fromCase(BYTES(), new byte[] {0, 1, 2, 3, 4}, new byte[] {0, 1, 2, 3, 4})
+ .fail(BOOLEAN(), true)
+ //
+ .fromCase(BINARY(2), DEFAULT_BINARY, DEFAULT_BINARY)
+ .fromCase(VARBINARY(3), DEFAULT_VARBINARY, DEFAULT_VARBINARY)
+ .fromCase(BYTES(), DEFAULT_BYTES, DEFAULT_BYTES)
// Not supported - no fix
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_NEGATIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
@@ -239,32 +319,34 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// RAW supported - check CastFunctionMiscITCase
.build(),
CastTestSpecBuilder.testCastTo(VARBINARY(4))
+ .fromCase(VARBINARY(5), null, null)
.fromCase(CHAR(3), "foo", new byte[] {102, 111, 111})
.fromCase(VARCHAR(5), "Flink", new byte[] {70, 108, 105, 110, 107})
// https://issues.apache.org/jira/browse/FLINK-24419 - not trimmed to 2
// bytes
.fromCase(STRING(), "Apache", new byte[] {65, 112, 97, 99, 104, 101})
- // Not supported
- // BOOLEAN
- .fromCase(BINARY(2), new byte[] {0, 1}, new byte[] {0, 1})
- .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, new byte[] {0, 1, 2})
- .fromCase(BYTES(), new byte[] {0, 1, 2, 3, 4}, new byte[] {0, 1, 2, 3, 4})
// Not supported - no fix
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(BOOLEAN(), true)
+ //
+ .fromCase(BINARY(2), DEFAULT_BINARY, DEFAULT_BINARY)
+ .fromCase(VARBINARY(3), DEFAULT_VARBINARY, DEFAULT_VARBINARY)
+ .fromCase(BYTES(), DEFAULT_BYTES, DEFAULT_BYTES)
+ // Not supported - no fix
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_NEGATIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
@@ -272,69 +354,75 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// RAW supported - check CastFunctionMiscITCase
.build(),
CastTestSpecBuilder.testCastTo(BYTES())
+ .fromCase(BYTES(), null, null)
.fromCase(CHAR(3), "foo", new byte[] {102, 111, 111})
.fromCase(VARCHAR(5), "Flink", new byte[] {70, 108, 105, 110, 107})
.fromCase(STRING(), "Apache", new byte[] {65, 112, 97, 99, 104, 101})
// Not supported - no fix
- // BOOLEAN
- .fromCase(BINARY(2), new byte[] {0, 1}, new byte[] {0, 1})
- .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, new byte[] {0, 1, 2})
- .fromCase(BYTES(), new byte[] {0, 1, 2, 3, 4}, new byte[] {0, 1, 2, 3, 4})
+ .fail(BOOLEAN(), true)
+ //
+ .fromCase(BINARY(2), DEFAULT_BINARY, DEFAULT_BINARY)
+ .fromCase(VARBINARY(3), DEFAULT_VARBINARY, DEFAULT_VARBINARY)
+ .fromCase(BYTES(), DEFAULT_BYTES, DEFAULT_BYTES)
// Not supported - no fix
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_NEGATIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
//
// RAW supported - check CastFunctionMiscITCase
.build(),
- CastTestSpecBuilder.testCastTo(DECIMAL(4, 3))
+ CastTestSpecBuilder.testCastTo(DECIMAL(5, 3))
+ .fromCase(DECIMAL(10, 2), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
- .fromCase(STRING(), "1.234", new BigDecimal("1.234".toCharArray(), 0, 5))
- .fromCase(BOOLEAN(), true, new BigDecimal("1.000".toCharArray(), 0, 5))
- .fromCase(BOOLEAN(), false, new BigDecimal("0.000".toCharArray(), 0, 5))
+ .fromCase(STRING(), "1.234", new BigDecimal("1.234"))
+ .fromCase(STRING(), "1.2", new BigDecimal("1.200"))
+ .fromCase(BOOLEAN(), true, new BigDecimal("1.000"))
+ .fromCase(BOOLEAN(), false, new BigDecimal("0.000"))
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, new BigDecimal("9.870"))
- .fromCase(TINYINT(), -125, null)
- .fromCase(SMALLINT(), 32767, null)
- .fromCase(INT(), -12345678, null)
- .fromCase(BIGINT(), 1234567891234L, null)
- .fromCase(FLOAT(), -123.456, null)
- .fromCase(DOUBLE(), 12345.67890, null)
+ .fromCase(TINYINT(), -1, new BigDecimal("-1.000"))
+ .fromCase(SMALLINT(), 3, new BigDecimal("3.000"))
+ .fromCase(INT(), 42, new BigDecimal("42.000"))
+ .fromCase(BIGINT(), 8, new BigDecimal("8.000"))
+ .fromCase(FLOAT(), -12.345, new BigDecimal("-12.345"))
+ .fromCase(DOUBLE(), 12.678, new BigDecimal("12.678"))
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(TINYINT())
+ .fromCase(TINYINT(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
@@ -344,39 +432,44 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, (byte) 1)
.fromCase(BOOLEAN(), false, (byte) 0)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, (byte) 9)
// https://issues.apache.org/jira/browse/FLINK-24420 - Check out of range
// instead of overflow
.fromCase(DECIMAL(10, 3), 9123.87, (byte) -93)
- .fromCase(TINYINT(), -125, (byte) -125)
+ .fromCase(TINYINT(), DEFAULT_POSITIVE_TINY_INT, DEFAULT_POSITIVE_TINY_INT)
+ .fromCase(TINYINT(), DEFAULT_NEGATIVE_TINY_INT, DEFAULT_NEGATIVE_TINY_INT)
.fromCase(SMALLINT(), 32, (byte) 32)
- .fromCase(SMALLINT(), 32767, (byte) -1)
+ .fromCase(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT, (byte) 57)
+ .fromCase(SMALLINT(), DEFAULT_NEGATIVE_SMALL_INT, (byte) -57)
.fromCase(INT(), -12, (byte) -12)
- .fromCase(INT(), -12345678, (byte) -78)
- .fromCase(BIGINT(), 123, (byte) 123)
- .fromCase(BIGINT(), 1234567891234L, (byte) 34)
- .fromCase(FLOAT(), -123.456, (byte) -123)
- .fromCase(FLOAT(), 128.456, (byte) -128)
- .fromCase(DOUBLE(), 123.4567890, (byte) 123)
- .fromCase(DOUBLE(), 12345.67890, (byte) 57)
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, (byte) -121)
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, (byte) 121)
+ .fromCase(BIGINT(), DEFAULT_POSITIVE_BIGINT, (byte) 53)
+ .fromCase(BIGINT(), DEFAULT_NEGATIVE_BIGINT, (byte) -53)
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, (byte) 123)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, (byte) -123)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, (byte) 123)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, (byte) -123)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(SMALLINT())
+ .fromCase(SMALLINT(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
@@ -386,33 +479,48 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, (short) 1)
.fromCase(BOOLEAN(), false, (short) 0)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, (short) 9)
// https://issues.apache.org/jira/browse/FLINK-24420 - Check out of range
// instead of overflow
.fromCase(DECIMAL(10, 3), 91235.87, (short) 25699)
- .fromCase(TINYINT(), -125, (short) -125)
- .fromCase(SMALLINT(), 32, (short) 32)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ (short) DEFAULT_POSITIVE_TINY_INT)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ (short) DEFAULT_NEGATIVE_TINY_INT)
+ .fromCase(
+ SMALLINT(), DEFAULT_POSITIVE_SMALL_INT, DEFAULT_POSITIVE_SMALL_INT)
+ .fromCase(
+ SMALLINT(), DEFAULT_NEGATIVE_SMALL_INT, DEFAULT_NEGATIVE_SMALL_INT)
.fromCase(SMALLINT(), 32780, (short) -32756)
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, (short) -10617)
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, (short) 10617)
.fromCase(INT(), -12, (short) -12)
- .fromCase(INT(), -12345678, (short) -24910)
.fromCase(BIGINT(), 123, (short) 123)
- .fromCase(BIGINT(), 1234567891234L, (short) 2338)
- .fromCase(FLOAT(), -123.456, (short) -123)
+ .fromCase(BIGINT(), DEFAULT_POSITIVE_BIGINT, (short) 7221)
+ .fromCase(BIGINT(), DEFAULT_NEGATIVE_BIGINT, (short) -7221)
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, (short) 123)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, (short) -123)
.fromCase(FLOAT(), 123456.78, (short) -7616)
- .fromCase(DOUBLE(), 123.4567890, (short) 123)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, (short) 123)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, (short) -123)
.fromCase(DOUBLE(), 123456.7890, (short) -7616)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
@@ -429,37 +537,57 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, 1)
.fromCase(BOOLEAN(), false, 0)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, 9)
// https://issues.apache.org/jira/browse/FLINK-24420 - Check out of range
// instead of overflow
.fromCase(DECIMAL(20, 3), 3276913443134.87, -146603714)
- .fromCase(TINYINT(), -125, -125)
- .fromCase(SMALLINT(), 32, 32)
- .fromCase(INT(), -12345678, -12345678)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ Integer.valueOf(DEFAULT_POSITIVE_TINY_INT))
+ .fromCase(
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ Integer.valueOf(DEFAULT_NEGATIVE_TINY_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_POSITIVE_SMALL_INT,
+ Integer.valueOf(DEFAULT_POSITIVE_SMALL_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_NEGATIVE_SMALL_INT,
+ Integer.valueOf(DEFAULT_NEGATIVE_SMALL_INT))
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, DEFAULT_POSITIVE_INT)
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, DEFAULT_NEGATIVE_INT)
.fromCase(BIGINT(), 123, 123)
- .fromCase(BIGINT(), 1234567891234L, 1912277282)
- .fromCase(FLOAT(), -123.456, -123)
+ .fromCase(BIGINT(), DEFAULT_POSITIVE_BIGINT, -539222987)
+ .fromCase(BIGINT(), DEFAULT_NEGATIVE_BIGINT, 539222987)
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, 123)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, -123)
.fromCase(FLOAT(), 9234567891.12, 644633299)
- .fromCase(DOUBLE(), 123.4567890, 123)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, 123)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, -123)
.fromCase(DOUBLE(), 9234567891.12345, 644633299)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(BIGINT())
+ .fromCase(BIGINT(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
@@ -469,34 +597,54 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, 1L)
.fromCase(BOOLEAN(), false, 0L)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, 9L)
.fromCase(DECIMAL(20, 3), 3276913443134.87, 3276913443134L)
- .fromCase(TINYINT(), -125, -125L)
- .fromCase(SMALLINT(), 32, 32L)
- .fromCase(INT(), -12345678, -12345678L)
- .fromCase(BIGINT(), 1234567891234L, 1234567891234L)
- .fromCase(FLOAT(), -123.456, -123L)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ Long.valueOf(DEFAULT_POSITIVE_TINY_INT))
+ .fromCase(
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ Long.valueOf(DEFAULT_NEGATIVE_TINY_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_POSITIVE_SMALL_INT,
+ Long.valueOf(DEFAULT_POSITIVE_SMALL_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_NEGATIVE_SMALL_INT,
+ Long.valueOf(DEFAULT_NEGATIVE_SMALL_INT))
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, Long.valueOf(DEFAULT_POSITIVE_INT))
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, Long.valueOf(DEFAULT_NEGATIVE_INT))
+ .fromCase(BIGINT(), DEFAULT_POSITIVE_BIGINT, DEFAULT_POSITIVE_BIGINT)
+ .fromCase(BIGINT(), DEFAULT_NEGATIVE_BIGINT, DEFAULT_NEGATIVE_BIGINT)
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, 123L)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, -123L)
.fromCase(FLOAT(), 9234567891.12, 9234567891L)
- .fromCase(DOUBLE(), 123.4567890, 123L)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, 123L)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, -123L)
.fromCase(DOUBLE(), 9234567891.12345, 9234567891L)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(FLOAT())
+ .fromCase(FLOAT(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
@@ -506,37 +654,63 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, 1.0f)
.fromCase(BOOLEAN(), false, 0.0f)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, 9.87f)
// https://issues.apache.org/jira/browse/FLINK-24420 - Check out of range
// instead of overflow
.fromCase(DECIMAL(4, 3), 9.87, 9.87f)
.fromCase(DECIMAL(20, 3), 3276913443134.87, 3.27691351E12f)
- .fromCase(TINYINT(), -125, -125f)
- .fromCase(SMALLINT(), 32, 32f)
- .fromCase(INT(), -12345678, -12345678f)
- .fromCase(BIGINT(), 1234567891234L, 1234567891234f)
- .fromCase(FLOAT(), -123.456, -123.456f)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ Float.valueOf(DEFAULT_POSITIVE_TINY_INT))
+ .fromCase(
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ Float.valueOf(DEFAULT_NEGATIVE_TINY_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_POSITIVE_SMALL_INT,
+ Float.valueOf(DEFAULT_POSITIVE_SMALL_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_NEGATIVE_SMALL_INT,
+ Float.valueOf(DEFAULT_NEGATIVE_SMALL_INT))
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, Float.valueOf(DEFAULT_POSITIVE_INT))
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, Float.valueOf(DEFAULT_NEGATIVE_INT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_POSITIVE_BIGINT,
+ Float.valueOf(DEFAULT_POSITIVE_BIGINT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_NEGATIVE_BIGINT,
+ Float.valueOf(DEFAULT_NEGATIVE_BIGINT))
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, DEFAULT_POSITIVE_FLOAT)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, DEFAULT_NEGATIVE_FLOAT)
.fromCase(FLOAT(), 9234567891.12, 9234567891.12f)
- .fromCase(DOUBLE(), 123.4567890, 123.45679f)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, 123.456789f)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, -123.456789f)
.fromCase(DOUBLE(), 1239234567891.1234567891234, 1.23923451E12f)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(DOUBLE())
+ .fromCase(DOUBLE(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "Apache", null)
@@ -546,34 +720,64 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(BOOLEAN(), true, 1.0d)
.fromCase(BOOLEAN(), false, 0.0d)
// Not supported - no fix
- // BINARY
- // VARBINARY
- // BYTES
+ .fail(BINARY(2), DEFAULT_BINARY)
+ .fail(VARBINARY(5), DEFAULT_VARBINARY)
+ .fail(BYTES(), DEFAULT_BYTES)
+ //
.fromCase(DECIMAL(4, 3), 9.87, 9.87d)
.fromCase(DECIMAL(20, 3), 3276913443134.87, 3.27691344313487E12d)
- .fromCase(TINYINT(), -125, -125d)
- .fromCase(SMALLINT(), 32, 32d)
- .fromCase(INT(), -12345678, -12345678d)
- .fromCase(BIGINT(), 1234567891234L, 1234567891234d)
- .fromCase(FLOAT(), -123.456, -123.456d)
+ .fromCase(
+ DECIMAL(30, 20),
+ new BigDecimal("123456789.123456789123456789"),
+ 1.2345678912345679E8d)
+ .fromCase(
+ TINYINT(),
+ DEFAULT_POSITIVE_TINY_INT,
+ Double.valueOf(DEFAULT_POSITIVE_TINY_INT))
+ .fromCase(
+ TINYINT(),
+ DEFAULT_NEGATIVE_TINY_INT,
+ Double.valueOf(DEFAULT_NEGATIVE_TINY_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_POSITIVE_SMALL_INT,
+ Double.valueOf(DEFAULT_POSITIVE_SMALL_INT))
+ .fromCase(
+ SMALLINT(),
+ DEFAULT_NEGATIVE_SMALL_INT,
+ Double.valueOf(DEFAULT_NEGATIVE_SMALL_INT))
+ .fromCase(INT(), DEFAULT_POSITIVE_INT, Double.valueOf(DEFAULT_POSITIVE_INT))
+ .fromCase(INT(), DEFAULT_NEGATIVE_INT, Double.valueOf(DEFAULT_NEGATIVE_INT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_POSITIVE_BIGINT,
+ Double.valueOf(DEFAULT_POSITIVE_BIGINT))
+ .fromCase(
+ BIGINT(),
+ DEFAULT_NEGATIVE_BIGINT,
+ Double.valueOf(DEFAULT_NEGATIVE_BIGINT))
+ .fromCase(FLOAT(), DEFAULT_POSITIVE_FLOAT, 123.456d)
+ .fromCase(FLOAT(), DEFAULT_NEGATIVE_FLOAT, -123.456)
.fromCase(FLOAT(), 9234567891.12, 9234567891.12d)
- .fromCase(DOUBLE(), 123.4567890, 123.456789d)
+ .fromCase(DOUBLE(), DEFAULT_POSITIVE_DOUBLE, DEFAULT_POSITIVE_DOUBLE)
+ .fromCase(DOUBLE(), DEFAULT_NEGATIVE_DOUBLE, DEFAULT_NEGATIVE_DOUBLE)
.fromCase(DOUBLE(), 1239234567891.1234567891234, 1.2392345678911235E12d)
// Not supported - no fix
- // DATE
- // TIME
- // TIMESTAMP
+ .fail(DATE(), DEFAULT_DATE)
+ .fail(TIME(), DEFAULT_TIME)
+ .fail(TIMESTAMP(), DEFAULT_TIMESTAMP)
// TIMESTAMP_WITH_TIME_ZONE
- // TIMESTAMP_LTZ
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP_LTZ)
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(DATE())
+ .fromCase(DATE(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "123", LocalDate.of(123, 1, 1))
@@ -584,51 +788,42 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
LocalDate.of(2021, 9, 27))
.fromCase(STRING(), "2021/09/27", null)
// Not supported - no fix
- // BOOLEAN
+ .fail(BOOLEAN(), true)
// BINARY
// VARBINARY
// BYTES
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- .fromCase(DATE(), LocalDate.parse("2021-09-24"), LocalDate.of(2021, 9, 24))
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_POSITIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ //
+ .fromCase(DATE(), DEFAULT_DATE, DEFAULT_DATE)
// Not supported - no fix
- // TIME
- .fromCase(
- TIMESTAMP(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalDate.of(2021, 9, 24))
- .fromCase(
- TIMESTAMP(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalDate.of(2021, 9, 24))
+ .fail(TIME(), DEFAULT_TIME)
+ //
+ .fromCase(TIMESTAMP(), DEFAULT_TIMESTAMP, LocalDate.of(2021, 9, 24))
+ .fromCase(TIMESTAMP(4), DEFAULT_TIMESTAMP, LocalDate.of(2021, 9, 24))
// https://issues.apache.org/jira/browse/FLINK-20869
// TIMESTAMP_WITH_TIME_ZONE
// https://issues.apache.org/jira/browse/FLINK-24422 - Accept only Instant
- .fromCase(
- TIMESTAMP_LTZ(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalDate.of(2021, 9, 24))
- .fromCase(
- TIMESTAMP_LTZ(),
- fromLocalTZ("2021-09-24T18:34:56.123456"),
- LocalDate.of(2021, 9, 25))
+ .fromCase(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP, LocalDate.of(2021, 9, 24))
+ .fromCase(TIMESTAMP_LTZ(), DEFAULT_TIMESTAMP, LocalDate.of(2021, 9, 24))
// Not supported - no fix
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(TIME())
+ .fromCase(TIME(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "123", LocalTime.of(23, 0, 0))
@@ -640,53 +835,41 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.fromCase(STRING(), "12:34:56.123456789", LocalTime.of(12, 34, 56, 0))
.fromCase(STRING(), "2021-09-27 12:34:56.123456789", null)
// Not supported - no fix
- // BOOLEAN
+ .fail(BOOLEAN(), true)
// BINARY
// VARBINARY
// BYTES
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- // DATE
- .fromCase(
- TIME(5),
- LocalTime.parse("12:34:56.1234567"),
- LocalTime.of(12, 34, 56, 0))
- .fromCase(
- TIMESTAMP(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalTime.of(12, 34, 56, 0))
- .fromCase(
- TIMESTAMP(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalTime.of(12, 34, 56, 0))
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_POSITIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ .fail(DATE(), DEFAULT_DATE)
+ //
+ .fromCase(TIME(5), DEFAULT_TIME, LocalTime.of(12, 34, 56, 0))
+ .fromCase(TIMESTAMP(), DEFAULT_TIMESTAMP, LocalTime.of(12, 34, 56, 0))
+ .fromCase(TIMESTAMP(4), DEFAULT_TIMESTAMP, LocalTime.of(12, 34, 56, 0))
// https://issues.apache.org/jira/browse/FLINK-20869
// TIMESTAMP_WITH_TIME_ZONE
// https://issues.apache.org/jira/browse/FLINK-24422 - Accept only Instant
+ .fromCase(TIMESTAMP_LTZ(4), DEFAULT_TIMESTAMP, LocalTime.of(12, 34, 56, 0))
.fromCase(
- TIMESTAMP_LTZ(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
- LocalTime.of(12, 34, 56, 0))
- .fromCase(
- TIMESTAMP_LTZ(4),
- fromLocalTZ("2021-09-24T22:34:56.123456"),
- LocalTime.of(7, 54, 56, 0))
+ TIMESTAMP_LTZ(4), DEFAULT_TIMESTAMP_LTZ, LocalTime.of(7, 54, 56, 0))
// Not supported - no fix
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(TIMESTAMP(9))
+ .fromCase(TIMESTAMP(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "123", null)
@@ -697,37 +880,37 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
"2021-09-27 12:34:56.123456789",
LocalDateTime.of(2021, 9, 27, 12, 34, 56, 123456789))
// Not supported - no fix
- // BOOLEAN
+ .fail(BOOLEAN(), true)
// BINARY
// VARBINARY
// BYTES
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
- .fromCase(
- DATE(),
- LocalDate.parse("2021-09-24"),
- LocalDateTime.of(2021, 9, 24, 0, 0, 0, 0))
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_POSITIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ //
+ .fromCase(DATE(), DEFAULT_DATE, LocalDateTime.of(2021, 9, 24, 0, 0, 0, 0))
// https://issues.apache.org/jira/browse/FLINK-17224 Fractional seconds are
// lost
// https://issues.apache.org/jira/browse/FLINK-24423 Continue using EPOCH
// date or use 0 for the year?
.fromCase(
- TIME(5),
- LocalTime.parse("12:34:56.1234567"),
- LocalDateTime.of(1970, 1, 1, 12, 34, 56, 0))
+ TIME(5), DEFAULT_TIME, LocalDateTime.of(1970, 1, 1, 12, 34, 56, 0))
.fromCase(
TIMESTAMP(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123456000))
.fromCase(
+ TIMESTAMP(8),
+ DEFAULT_TIMESTAMP,
+ LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123456700))
+ .fromCase(
TIMESTAMP(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123400000))
// https://issues.apache.org/jira/browse/FLINK-20869
@@ -736,22 +919,23 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// https://issues.apache.org/jira/browse/FLINK-24422 - Accept only Instant
.fromCase(
TIMESTAMP_LTZ(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123400000))
.fromCase(
TIMESTAMP_LTZ(4),
- fromLocalTZ("2021-09-24T22:34:56.123456"),
+ DEFAULT_TIMESTAMP_LTZ,
LocalDateTime.of(2021, 9, 25, 7, 54, 56, 123400000))
// Not supported - no fix
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
// RAW
.build(),
CastTestSpecBuilder.testCastTo(TIMESTAMP_LTZ(9))
+ .fromCase(TIMESTAMP_LTZ(), null, null)
.fromCase(CHAR(3), "foo", null)
.fromCase(VARCHAR(5), "Flink", null)
.fromCase(STRING(), "123", null)
@@ -772,20 +956,21 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
fromLocalToUTC(LocalDateTime.of(2021, 9, 27, 12, 34, 56, 0)))
// Not supported - no fix
- // BOOLEAN
+ .fail(BOOLEAN(), true)
// BINARY
// VARBINARY
// BYTES
- // DECIMAL
- // TINYINT
- // SMALLINT
- // INT
- // BIGINT
- // FLOAT
- // DOUBLE
+ .fail(DECIMAL(5, 3), 12.345)
+ .fail(TINYINT(), DEFAULT_POSITIVE_TINY_INT)
+ .fail(SMALLINT(), DEFAULT_POSITIVE_SMALL_INT)
+ .fail(INT(), DEFAULT_POSITIVE_INT)
+ .fail(BIGINT(), DEFAULT_POSITIVE_BIGINT)
+ .fail(FLOAT(), DEFAULT_POSITIVE_FLOAT)
+ .fail(DOUBLE(), DEFAULT_POSITIVE_DOUBLE)
+ //
.fromCase(
DATE(),
- LocalDate.parse("2021-09-24"),
+ DEFAULT_DATE,
fromLocalToUTC(LocalDateTime.of(2021, 9, 24, 0, 0, 0, 0)))
// https://issues.apache.org/jira/browse/FLINK-17224 Fractional seconds are
@@ -794,16 +979,21 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// date or use 0 for the year?
.fromCase(
TIME(5),
- LocalTime.parse("12:34:56.1234567"),
+ DEFAULT_TIME,
fromLocalToUTC(LocalDateTime.of(1970, 1, 1, 12, 34, 56, 0)))
.fromCase(
TIMESTAMP(),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
fromLocalToUTC(
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123456000)))
.fromCase(
+ TIMESTAMP(8),
+ DEFAULT_TIMESTAMP,
+ fromLocalToUTC(
+ LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123456700)))
+ .fromCase(
TIMESTAMP(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
fromLocalToUTC(
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123400000)))
@@ -813,17 +1003,17 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// https://issues.apache.org/jira/browse/FLINK-24422 - Accept only Instant
.fromCase(
TIMESTAMP_LTZ(4),
- LocalDateTime.parse("2021-09-24T12:34:56.123456"),
+ DEFAULT_TIMESTAMP,
fromLocalToUTC(
LocalDateTime.of(2021, 9, 24, 12, 34, 56, 123400000)))
.fromCase(
TIMESTAMP_LTZ(4),
- fromLocalTZ("2021-09-24T22:34:56.123456"),
+ DEFAULT_TIMESTAMP_LTZ,
fromLocalToUTC(LocalDateTime.of(2021, 9, 25, 7, 54, 56, 123400000)))
// Not supported - no fix
- // INTERVAL_YEAR_MONTH
- // INTERVAL_DAY_TIME
- // ARRAY
+ .fail(INTERVAL(YEAR(), MONTH()), DEFAULT_INTERVAL_YEAR)
+ .fail(INTERVAL(DAY(), SECOND()), DEFAULT_INTERVAL_DAY)
+ .fail(ARRAY(INT()), DEFAULT_ARRAY)
// MULTISET
// MAP
// ROW
@@ -852,6 +1042,7 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// https://issues.apache.org/jira/browse/FLINK-24403 allow cast from string
// .fromCase(STRING(), "'+01-05'".resultsIn(Period.of(0, 17, 0)
.testCastTo(INTERVAL(MONTH()))
+ .fromCase(INTERVAL(MONTH()), null, null)
.fromCase(INTERVAL(YEAR()), 0, Period.of(0, 0, 0))
.fromCase(INTERVAL(YEAR()), 11, Period.of(0, 11, 0))
.fromCase(INTERVAL(YEAR()), 84, Period.of(0, 84, 0))
@@ -869,24 +1060,25 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
// https://issues.apache.org/jira/browse/FLINK-24428
// .build()
CastTestSpecBuilder.testCastTo(ARRAY(INT()))
+ .fromCase(ARRAY(INT()), null, null)
// https://issues.apache.org/jira/browse/FLINK-17321
// .fromCase(ARRAY(STRING()), new String[] {'1', '2', '3'}, new Integer[]
// {1, 2, 3})
// https://issues.apache.org/jira/browse/FLINK-24425 Cast from corresponding
// single type
- // .fromCase(INT(), 10, new int[] {10})
+ // .fromCase(INT(), DEFAULT_POSITIVE_INT, new int[] {DEFAULT_POSITIVE_INT})
.fromCase(ARRAY(INT()), new int[] {1, 2, 3}, new Integer[] {1, 2, 3})
.build(),
CastTestSpecBuilder.testCastTo(ARRAY(STRING().nullable()))
.fromCase(
- ARRAY(TIMESTAMP().nullable()),
+ ARRAY(TIMESTAMP(4).nullable()),
new LocalDateTime[] {
LocalDateTime.parse("2021-09-24T12:34:56.123456"),
null,
LocalDateTime.parse("2021-09-24T14:34:56.123456")
},
new String[] {
- "2021-09-24 12:34:56.123456", null, "2021-09-24 14:34:56.123456"
+ "2021-09-24 12:34:56.1234", null, "2021-09-24 14:34:56.1234"
})
.build(),
CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().nullable()))
@@ -898,8 +1090,9 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().notNull()))
.fromCase(ARRAY(INT().notNull()), new Integer[] {1, 2}, new Long[] {1L, 2L})
.build()
+ //
+ // Cast to structuredTypes
// https://issues.apache.org/jira/browse/FLINK-17321
- // ARRAY
// MULTISET
// MAP
// RAW
@@ -907,11 +1100,80 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
);
}
- private static class CastTestSpecBuilder {
+ public static List<TestSpec> decimalCasts() {
+ return Collections.singletonList(
+ CastTestSpecBuilder.testCastTo(DECIMAL(8, 4))
+ .fromCase(STRING(), null, null)
+ // rounding
+ .fromCase(DOUBLE(), 3.123456, new BigDecimal("3.1235"))
+ .fromCase(DECIMAL(10, 8), 12.34561234, new BigDecimal("12.3456"))
+ // out of precision/scale bounds
+ .fromCase(INT(), 12345, null)
+ .fromCase(FLOAT(), 12345.678912, null)
+ .fromCase(STRING(), 12345.6789, null)
+ .build());
+ }
+
+ public static List<TestSpec> numericBounds() {
+ return Arrays.asList(
+ CastTestSpecBuilder.testCastTo(TINYINT())
+ .fromCase(TINYINT(), Byte.MIN_VALUE, Byte.MIN_VALUE)
+ .fromCase(TINYINT(), Byte.MAX_VALUE, Byte.MAX_VALUE)
+ .fromCase(TINYINT(), Byte.MIN_VALUE - 1, Byte.MAX_VALUE)
+ .fromCase(TINYINT(), Byte.MAX_VALUE + 1, Byte.MIN_VALUE)
+ .build(),
+ CastTestSpecBuilder.testCastTo(SMALLINT())
+ .fromCase(SMALLINT(), Short.MIN_VALUE, Short.MIN_VALUE)
+ .fromCase(SMALLINT(), Short.MAX_VALUE, Short.MAX_VALUE)
+ .fromCase(SMALLINT(), Short.MIN_VALUE - 1, Short.MAX_VALUE)
+ .fromCase(SMALLINT(), Short.MAX_VALUE + 1, Short.MIN_VALUE)
+ .build(),
+ CastTestSpecBuilder.testCastTo(INT())
+ .fromCase(INT(), Integer.MIN_VALUE, Integer.MIN_VALUE)
+ .fromCase(INT(), Integer.MAX_VALUE, Integer.MAX_VALUE)
+ .fromCase(INT(), Integer.MIN_VALUE - 1, Integer.MAX_VALUE)
+ .fromCase(INT(), Integer.MAX_VALUE + 1, Integer.MIN_VALUE)
+ .build(),
+ CastTestSpecBuilder.testCastTo(BIGINT())
+ .fromCase(BIGINT(), Long.MIN_VALUE, Long.MIN_VALUE)
+ .fromCase(BIGINT(), Long.MAX_VALUE, Long.MAX_VALUE)
+ .fromCase(BIGINT(), Long.MIN_VALUE - 1, Long.MAX_VALUE)
+ .fromCase(BIGINT(), Long.MAX_VALUE + 1, Long.MIN_VALUE)
+ .build(),
+ CastTestSpecBuilder.testCastTo(FLOAT())
+ .fromCase(DOUBLE(), -1.7976931348623157E308d, Float.NEGATIVE_INFINITY)
+ .build(),
+ CastTestSpecBuilder.testCastTo(DECIMAL(38, 0))
+ .fromCase(TINYINT(), Byte.MIN_VALUE - 1, new BigDecimal(Byte.MIN_VALUE - 1))
+ .fromCase(TINYINT(), Byte.MAX_VALUE + 1, new BigDecimal(Byte.MAX_VALUE + 1))
+ .fromCase(
+ SMALLINT(),
+ Short.MIN_VALUE - 1,
+ new BigDecimal(Short.MIN_VALUE - 1))
+ .fromCase(
+ SMALLINT(),
+ Short.MAX_VALUE + 1,
+ new BigDecimal(Short.MAX_VALUE + 1))
+ .fromCase(
+ INT(), Integer.MIN_VALUE - 1, new BigDecimal(Integer.MIN_VALUE - 1))
+ .fromCase(
+ INT(), Integer.MAX_VALUE + 1, new BigDecimal(Integer.MAX_VALUE + 1))
+ .fromCase(BIGINT(), Long.MIN_VALUE - 1, new BigDecimal(Long.MIN_VALUE - 1))
+ .fromCase(BIGINT(), Long.MAX_VALUE + 1, new BigDecimal(Long.MAX_VALUE + 1))
+ .build(),
+ CastTestSpecBuilder.testCastTo(DECIMAL(38, 32))
+ .fromCase(FLOAT(), -Float.MAX_VALUE, null)
+ .fromCase(FLOAT(), Float.MAX_VALUE, null)
+ .fromCase(DOUBLE(), -Double.MAX_VALUE, null)
+ .fromCase(DOUBLE(), Double.MAX_VALUE, null)
+ .build());
+ }
+
+ static class CastTestSpecBuilder {
private TestSpec testSpec;
private DataType targetType;
private final List<Object> columnData = new ArrayList<>();
- private final List<AbstractDataType<?>> columnTypes = new ArrayList<>();
+ private final List<DataType> columnTypes = new ArrayList<>();
private final List<Object> expectedValues = new ArrayList<>();
private final List<TestType> testTypes = new ArrayList<>();
@@ -930,8 +1192,7 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
return tsb;
}
- private CastTestSpecBuilder fromCase(
- AbstractDataType<?> dataType, Object src, Object target) {
+ private CastTestSpecBuilder fromCase(DataType dataType, Object src, Object target) {
this.testTypes.add(TestType.RESULT);
this.columnTypes.add(dataType);
this.columnData.add(src);
@@ -939,8 +1200,21 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
return this;
}
- private CastTestSpecBuilder failTableApi(AbstractDataType<?> dataType, Object src) {
- this.testTypes.add(TestType.ERROR_TABLE_API);
+ private CastTestSpecBuilder failTableApi(DataType dataType, Object src) {
+ return fail(TestType.ERROR_TABLE_API, dataType, src);
+ }
+
+ private CastTestSpecBuilder failSQL(DataType dataType, Object src) {
+ return fail(TestType.ERROR_TABLE_API, dataType, src);
+ }
+
+ private CastTestSpecBuilder fail(DataType dataType, Object src) {
+ fail(TestType.ERROR_TABLE_API, dataType, src);
+ return fail(TestType.ERROR_SQL, dataType, src);
+ }
+
+ private CastTestSpecBuilder fail(TestType type, DataType dataType, Object src) {
+ this.testTypes.add(type);
this.columnTypes.add(dataType);
this.columnData.add(src);
return this;
@@ -952,16 +1226,28 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
int idxOffset = 0;
for (int i = 0; i < columnData.size(); i++) {
String colName = "f" + i;
+ LogicalType colType = columnTypes.get(i).getLogicalType();
+ String errorMsg;
switch (testTypes.get(i)) {
case ERROR_TABLE_API:
- testSpec.testTableApiValidationError(
- $(colName).cast(targetType),
- String.format(
- "Invalid function call:%ncast("
- + columnTypes.get(i).toString()
- + ", "
- + targetType.toString()
- + ")"));
+ errorMsg =
+ specificErrorMsg(
+ colType,
+ String.format(
+ "Invalid function call:%ncast("
+ + columnTypes.get(i).toString()
+ + ", "
+ + targetType.toString()
+ + ")"));
+ testSpec.testTableApiValidationError($(colName).cast(targetType), errorMsg);
+ idxOffset++;
+ break;
+ case ERROR_SQL:
+ errorMsg =
+ specificErrorMsg(
+ colType, "Cast function cannot convert value of type ");
+ testSpec.testSqlValidationError(
+ "CAST(" + colName + " AS " + targetType.toString() + ")", errorMsg);
idxOffset++;
break;
case RESULT:
@@ -979,6 +1265,18 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
.testResult(testSpecs.toArray(new ResultSpec[0]));
return testSpec;
}
+
+ private String specificErrorMsg(LogicalType colType, String defaultMsg) {
+ if (isTimestampLtzToNumeric(colType, targetType.getLogicalType())) {
+ return "The cast conversion from TIMESTAMP_LTZ type to NUMERIC type is not allowed.";
+ } else if (isNumericToTimestamp(colType, targetType.getLogicalType())) {
+ return "type is not allowed, it's recommended to use TO_TIMESTAMP";
+ } else if (isTimestampToNumeric(colType, targetType.getLogicalType())) {
+ return "type is not allowed, it's recommended to use UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead.";
+ } else {
+ return defaultMsg;
+ }
+ }
}
private static Instant fromLocalToUTC(LocalDateTime localDateTime) {
@@ -988,4 +1286,17 @@ public class CastFunctionITCase extends BuiltInFunctionTestBase {
private static Instant fromLocalTZ(String str) {
return LocalDateTime.parse(str).toInstant(TEST_OFFSET);
}
+
+ private static boolean isTimestampLtzToNumeric(LogicalType srcType, LogicalType trgType) {
+ return srcType.is(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)
+ && trgType.is(LogicalTypeFamily.NUMERIC);
+ }
+
+ private static boolean isNumericToTimestamp(LogicalType srcType, LogicalType trgType) {
+ return srcType.is(LogicalTypeFamily.NUMERIC) && trgType.is(LogicalTypeFamily.TIMESTAMP);
+ }
+
+ private static boolean isTimestampToNumeric(LogicalType srcType, LogicalType trgType) {
+ return srcType.is(LogicalTypeFamily.TIMESTAMP) && trgType.is(LogicalTypeFamily.NUMERIC);
+ }
}
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalCastTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalCastTest.scala
deleted file mode 100644
index 6b0b4c0..0000000
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalCastTest.scala
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.planner.expressions
-
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
-import org.apache.flink.types.Row
-
-import org.junit.Test
-
-import scala.util.Random
-
-class DecimalCastTest extends ExpressionTestBase {
-
- val rnd = new Random()
-
- @Test
- def testCastFromNumeric(): Unit = {
- def test(t: String, max: Any, min: Any, rV: Any): Unit = {
- def value(i: Any) = s"CAST($i AS $t)"
-
- testSqlApi(s"CAST(${value(null)} AS DECIMAL)", "null")
-
- testSqlApi(s"CAST(${value(0)} AS DECIMAL)", "0")
- testSqlApi(s"CAST(${value(12)} AS DECIMAL)", "12")
- testSqlApi(s"CAST(${value(-12)} AS DECIMAL)", "-12")
- testSqlApi(s"CAST(${value(max)} AS DECIMAL(20, 0))", max.toString)
- testSqlApi(s"CAST(${value(min)} AS DECIMAL(20, 0))", min.toString)
-
- testSqlApi(s"CAST(${value(rV)} AS DECIMAL(20, 0))", rV.toString)
-
- testSqlApi(s"CAST(${value(100)} AS DECIMAL(2, 0))", "null")
- }
-
- test("TINYINT", Byte.MaxValue, Byte.MinValue, rnd.nextInt().toByte)
- test("SMALLINT", Short.MaxValue, Short.MinValue, rnd.nextInt().toShort)
- test("INT", Int.MaxValue, Int.MinValue, rnd.nextInt())
- test("BIGINT", Long.MaxValue, Long.MinValue, rnd.nextLong())
- }
-
- @Test
- def testCastFromFloat(): Unit = {
- def test(t: String, max: Any, min: Any, rV: Any): Unit = {
- def value(i: Any) = s"CAST($i AS $t)"
-
- testSqlApi(s"CAST(${value(null)} AS DECIMAL)", "null")
-
- testSqlApi(s"CAST(${value(0)} AS DECIMAL)", "0")
- testSqlApi(s"CAST(${value(12.2)} AS DECIMAL)", "12")
- testSqlApi(s"CAST(${value(-12.2)} AS DECIMAL)", "-12")
- testSqlApi(s"CAST(${value(max)} AS DECIMAL(38, 0))", "null")
- testSqlApi(s"CAST(${value(min)} AS DECIMAL(38, 0))", "null")
-
- testSqlApi(s"CAST(${value(rV)} AS DECIMAL(38, 0))", rV.toString)
-
- testSqlApi(s"CAST(${value(100)} AS DECIMAL(2, 0))", "null")
-
- testSqlApi(s"CAST(${value(6.8242f)} AS DECIMAL(10, 4))", "6.8242")
-
- // Test Round HALF_UP
- testSqlApi(s"CAST(${value(6.8242f)} AS DECIMAL(10, 3))", "6.824")
- testSqlApi(s"CAST(${value(6.8247f)} AS DECIMAL(10, 3))", "6.825")
-
- testSqlApi(s"CAST(${value(6.82f)} AS DECIMAL(10, 5))", "6.82000")
- testSqlApi(s"CAST(${value(6.82f)} AS DECIMAL(5, 5))", "null")
- testSqlApi(s"CAST(${value(66.82f)} AS DECIMAL(5, 4))", "null")
- }
-
- test("FLOAT", Float.MaxValue, Float.MinValue, rnd.nextInt())
- test("DOUBLE", Double.MaxValue, Double.MinValue, rnd.nextInt())
- }
-
- @Test
- def testCastFromString(): Unit = {
- // CHAR AND VARCHAR
- testSqlApi(s"CAST(CAST(null AS VARCHAR) AS DECIMAL)", "null")
- testSqlApi(s"CAST(CAST(null AS CHAR) AS DECIMAL)", "null")
-
- testSqlApi(s"CAST('0' AS DECIMAL)", "0")
- testSqlApi(s"CAST(CAST('12.2' AS VARCHAR) AS DECIMAL)", "12")
- testSqlApi(s"CAST('-12.2' AS DECIMAL)", "-12")
-
- val rV = rnd.nextInt()
- testSqlApi(s"CAST('$rV' AS DECIMAL(38, 0))", rV.toString)
-
- testSqlApi(s"CAST('100' AS DECIMAL(2, 0))", "null")
- testSqlApi(s"CAST('x100' AS DECIMAL(2, 0))", "null")
- testSqlApi(s"CAST('100x' AS DECIMAL(2, 0))", "null")
- }
-
- @Test
- def testCastFromBoolean(): Unit = {
- testSqlApi(s"CAST(CAST(null AS BOOLEAN) AS DECIMAL)", "null")
- testSqlApi(s"CAST(true AS DECIMAL)", "1")
- testSqlApi(s"CAST(false AS DECIMAL)", "0")
- }
-
- @Test
- def testCastToNumeric(): Unit = {
- def test(t: String, max: Any, min: Any, rV: Any): Unit = {
- testSqlApi(s"CAST(${decimal_38_18(null)} AS $t)", "null")
-
- testSqlApi(s"CAST(${decimal_38_18(0)} AS $t)", "0")
- testSqlApi(s"CAST(${decimal_38_18(12)} AS $t)", "12")
- testSqlApi(s"CAST(${decimal_38_18(-12)} AS $t)", "-12")
- testSqlApi(s"CAST(${decimal_38_18(max)} AS $t)", max.toString)
- testSqlApi(s"CAST(${decimal_38_18(min)} AS $t)", min.toString)
- testSqlApi(s"CAST(${decimal_38_18(rV)} AS $t)", rV.toString)
-
- testSqlApi(s"CAST(${decimal_38_18(5.26)} AS $t)", "5")
- }
-
- test("TINYINT", Byte.MaxValue, Byte.MinValue, rnd.nextInt().toByte)
- test("SMALLINT", Short.MaxValue, Short.MinValue, rnd.nextInt().toShort)
- test("INT", Int.MaxValue, Int.MinValue, rnd.nextInt())
- test("BIGINT", Long.MaxValue, Long.MinValue, rnd.nextLong())
-
- // test cast overflow
-
- // 128 => -128
- testSqlApi(s"CAST(${decimal_38_18(Byte.MaxValue + 1)} AS TINYINT)", "-128")
-
- // 32768 => -32768
- testSqlApi(s"CAST(${decimal_38_18(Short.MaxValue + 1)} AS SMALLINT)", "-32768")
-
- // 2147483648 => -2147483648
- testSqlApi(s"CAST(${decimal_38_18(Int.MaxValue + 1L)} AS INT)", "-2147483648")
-
- // 9223372036854775808 => -9223372036854775808
- testSqlApi(
- s"CAST(${decimal_38_18(BigDecimal.apply(Long.MaxValue) + 1)} AS BIGINT)",
- "-9223372036854775808")
- }
-
- @Test
- def testCastToFloat(): Unit = {
- def test(t: String): Unit = {
- testSqlApi(s"CAST(${decimal_38_18(null)} AS $t)", "null")
- testSqlApi(s"CAST(${decimal_38_18(0)} AS $t)", "0.0")
- testSqlApi(s"CAST(${decimal_38_18(12.2)} AS $t)", "12.2")
- testSqlApi(s"CAST(${decimal_38_18(-12.2)} AS $t)", "-12.2")
- testSqlApi(s"CAST(${decimal_38_18(5.26)} AS $t)", "5.26")
- }
-
- test("FLOAT")
- test("DOUBLE")
- }
-
- @Test
- def testCastToString(): Unit = {
- def test(t: String): Unit = {
- testSqlApi(s"CAST(${decimal_38_2(null)} AS $t)", "null")
- testSqlApi(s"CAST(${decimal_38_2(0)} AS $t)", "0.00")
- testSqlApi(s"CAST(${decimal_38_2(12.2)} AS $t)", "12.20")
- testSqlApi(s"CAST(${decimal_38_2(-12.2)} AS $t)", "-12.20")
- testSqlApi(s"CAST(${decimal_38_2(5.26)} AS $t)", "5.26")
- }
-
- test("VARCHAR")
- test("CHAR") // current CHAR is same to VARCHAR
- }
-
- @Test
- def testCastToBoolean(): Unit = {
- testSqlApi(s"CAST(${decimal_38_2(null)} AS BOOLEAN)", "null")
- testSqlApi(s"CAST(${decimal_38_2(0)} AS BOOLEAN)", "false")
- testSqlApi(s"CAST(${decimal_38_2(1)} AS BOOLEAN)", "true")
- testSqlApi(s"CAST(${decimal_38_2(12.2)} AS BOOLEAN)", "true")
- testSqlApi(s"CAST(${decimal_38_2(-12.2)} AS BOOLEAN)", "true")
- }
-
- @Test
- def testCastToDecimal(): Unit = {
- testSqlApi(s"CAST(${decimal_38_2(null)} AS DECIMAL)", "null")
- testSqlApi(s"CAST(${decimal_38_2(0)} AS DECIMAL(1, 1))", "0.0")
- testSqlApi(s"CAST(${decimal_38_2(6.32)} AS DECIMAL(3, 2))", "6.32")
- testSqlApi(s"CAST(${decimal_38_2(236.2)} AS DECIMAL(2, 1))", "null")
-
- // Test Round HALF_UP
- testSqlApi(s"CAST(${decimal_38_2(5.22)} AS DECIMAL(2, 1))", "5.2")
- testSqlApi(s"CAST(${decimal_38_2(5.26)} AS DECIMAL(2, 1))", "5.3")
- }
-
- private def testUnsupportedCastTo(t: String): Unit = {
- expectedException.expectMessage(
- "Cast function cannot convert value of type DECIMAL(38, 2) to type")
- testSqlApi(s"CAST(${decimal_38_2(5.22)} AS $t)", "")
- }
-
- @Test
- def testCastToBinary(): Unit = {
- testUnsupportedCastTo("BINARY(4)")
- }
-
- @Test
- def testCastToVarBinary(): Unit = {
- testUnsupportedCastTo("VARBINARY")
- }
-
- @Test
- def testCastToDate(): Unit = {
- testUnsupportedCastTo("DATE")
- }
-
- @Test
- def testCastToTime(): Unit = {
- testUnsupportedCastTo("TIME")
- }
-
- @Test
- def testCastToTimestamp(): Unit = {
- expectedException.expectMessage(
- "The cast conversion from NUMERIC type to TIMESTAMP type is not allowed," +
- " it's recommended to use TO_TIMESTAMP(FROM_UNIXTIME(numeric_col)) instead," +
- " note the numeric is in seconds")
- testSqlApi(s"CAST(${decimal_38_2(5.22)} AS TIMESTAMP)", "")
- }
-
- @Test
- def testCastToTimestampLz(): Unit = {
- expectedException.expectMessage(
- "The cast conversion from NUMERIC type to TIMESTAMP_LTZ type is not allowed," +
- " it's recommended to use TO_TIMESTAMP_LTZ(numeric_col, precision) instead")
- testSqlApi(s"CAST(${decimal_38_2(5.22)} AS TIMESTAMP_LTZ)", "")
- }
-
- @Test
- def testCastToArray(): Unit = {
- testUnsupportedCastTo("ARRAY<INT>")
- }
-
- @Test
- def testCastToMultiSet(): Unit = {
- testUnsupportedCastTo("MULTISET<INT>")
- }
-
- @Test
- def testCastToMap(): Unit = {
- testUnsupportedCastTo("MAP<INT, INT>")
- }
-
- @Test
- def testCastToRow(): Unit = {
- testUnsupportedCastTo("ROW<i INT, j INT>")
- }
-
- private def testUnsupportedCastFrom(t: String): Unit = {
- expectedException.expectMessage(
- "Cast function cannot convert value of type")
- testSqlApi(s"CAST(CAST(NULL AS $t) AS DECIMAL(38, 2))", "")
- }
-
- @Test
- def testCastFromBinary(): Unit = {
- testUnsupportedCastFrom("BINARY(4)")
- }
-
- @Test
- def testCastFromVarBinary(): Unit = {
- testUnsupportedCastFrom("VARBINARY")
- }
-
- @Test
- def testCastFromDate(): Unit = {
- testUnsupportedCastFrom("DATE")
- }
-
- @Test
- def testCastFromTime(): Unit = {
- testUnsupportedCastFrom("TIME")
- }
-
- @Test
- def testCastFromTimestamp(): Unit = {
- expectedException.expectMessage(
- "The cast conversion from TIMESTAMP type to NUMERIC type is not allowed, " +
- "it's recommended to use UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead")
- testSqlApi(s"CAST(TIMESTAMP '2020-02-02' AS DECIMAL(38, 2))", "")
- }
-
- @Test
- def testCastFromTimestampLz(): Unit = {
- expectedException.expectMessage(
- "The cast conversion from TIMESTAMP_LTZ type to NUMERIC type is not allowed.")
- testSqlApi(s"CAST(CAST('2020-02-02' AS TIMESTAMP_LTZ) AS DECIMAL(38, 2))", "")
- }
-
- @Test
- def testCastFromArray(): Unit = {
- testUnsupportedCastFrom("ARRAY<INT>")
- }
-
- @Test
- def testCastFromMultiSet(): Unit = {
- testUnsupportedCastFrom("MULTISET<INT>")
- }
-
- @Test
- def testCastFromMap(): Unit = {
- testUnsupportedCastFrom("MAP<INT, INT>")
- }
-
- @Test
- def testCastFromRow(): Unit = {
- testUnsupportedCastFrom("ROW<i INT, j INT>")
- }
-
- def decimal_38_18(i: Any) = s"CAST('$i' AS DECIMAL(38, 18))"
- def decimal_38_2(i: Any) = s"CAST('$i' AS DECIMAL(38, 2))"
-
- override def testData: Row = new Row(0)
-
- override def typeInfo: RowTypeInfo = new RowTypeInfo()
-}
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
index 8478151..b9ed972 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
@@ -70,86 +70,6 @@ class DecimalTypeTest extends ExpressionTestBase {
Double.MinValue,
Double.MinValue.toString,
Double.MinValue.toString)
-
- testAllApis(
- Double.MinValue.cast(DataTypes.FLOAT),
- s"CAST(${Double.MinValue} AS FLOAT)",
- Float.NegativeInfinity.toString)
-
- testAllApis(
- Byte.MinValue.cast(DataTypes.TINYINT),
- s"CAST(${Byte.MinValue} AS TINYINT)",
- Byte.MinValue.toString)
-
- testAllApis(
- Byte.MinValue.cast(DataTypes.TINYINT) - 1.cast(DataTypes.TINYINT),
- s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
- Byte.MaxValue.toString)
-
- testAllApis(
- Short.MinValue.cast(DataTypes.SMALLINT),
- s"CAST(${Short.MinValue} AS SMALLINT)",
- Short.MinValue.toString)
-
- testAllApis(
- Int.MinValue.cast(DataTypes.INT) - 1,
- s"CAST(${Int.MinValue} AS INT) - 1",
- Int.MaxValue.toString)
-
- testAllApis(
- Long.MinValue.cast(DataTypes.BIGINT()),
- s"CAST(${Long.MinValue} AS BIGINT)",
- Long.MinValue.toString)
- }
-
- @Test
- def testDefaultDecimalCasting(): Unit = {
- // from String
- testTableApi(
- "123456789123456789123456789".cast(DataTypes.DECIMAL(38, 0)),
- "123456789123456789123456789")
-
- // from double
- testAllApis(
- 'f3.cast(DataTypes.DECIMAL(38, 0)),
- "CAST(f3 AS DECIMAL)",
- "4")
- }
-
- @Test
- def testDecimalCasting(): Unit = {
- testSqlApi(
- "CAST(f3 AS DECIMAL(10,2))",
- "4.20"
- )
-
- // to double
- testAllApis(
- 'f0.cast(DataTypes.DOUBLE),
- "CAST(f0 AS DOUBLE)",
- "1.2345678912345679E8")
-
- // to int
- testAllApis(
- 'f4.cast(DataTypes.INT),
- "CAST(f4 AS INT)",
- "123456789")
-
- // to long
- testAllApis(
- 'f4.cast(DataTypes.BIGINT()),
- "CAST(f4 AS BIGINT)",
- "123456789")
-
- testTableApi(
- BigDecimal("123456789.123456789123456789").cast(DataTypes.DOUBLE),
- "1.2345678912345679E8")
-
- // testing padding behaviour
- testSqlApi(
- "CAST(CAST(f67 AS DECIMAL(10, 5)) AS VARCHAR)",
- "1.00000"
- )
}
@Test
@@ -779,57 +699,6 @@ class DecimalTypeTest extends ExpressionTestBase {
}
@Test
- def testCast(): Unit = {
-
- // String, numeric/Decimal => Decimal
- testSqlApi(
- "cast(f48 as Decimal(8,4))",
- "3.1400")
-
- testSqlApi(
- "cast(f2 as Decimal(8,4))",
- "42.0000")
-
- testSqlApi(
- "cast(f3 as Decimal(8,4))",
- "4.2000")
-
- testSqlApi(
- "cast(f55 as Decimal(8,4))",
- "3.1400")
-
- // round up
- testSqlApi(
- "cast(f56 as Decimal(8,1))",
- "3.2")
-
- testSqlApi(
- "cast(f57 as Decimal(8,1))",
- "3.2")
-
- testSqlApi(
- "cast(f58 as Decimal(8,1))",
- "3.2")
-
- testSqlApi(
- "cast(f59 as Decimal(3,2))",
- "null")
-
- // Decimal => String, numeric
- testSqlApi(
- "cast(f60 as VARCHAR(64))",
- "1.99")
-
- testSqlApi(
- "cast(f61 as DOUBLE)",
- "1.99")
-
- testSqlApi(
- "cast(f62 as INT)",
- "1")
- }
-
- @Test
def testEquality(): Unit = {
// expressions that test equality.
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala
index bb34a40..aefaed2 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala
@@ -144,14 +144,10 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase {
testSqlApi("CASE WHEN 'a'='a' THEN 1 END", "1")
testSqlApi("CASE 2 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "bcd")
testSqlApi("CASE 1 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "a")
- testSqlApi("CASE 1 WHEN 1 THEN CAST ('a' as varchar(1)) WHEN 2 THEN " +
- "CAST ('bcd' as varchar(3)) END", "a")
testSqlApi("CASE f2 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "11")
testSqlApi("CASE f7 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "null")
testSqlApi("CASE 42 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "null")
testSqlApi("CASE 1 WHEN 1 THEN true WHEN 2 THEN false ELSE NULL END", "true")
-
- testSqlApi("CASE WHEN f2 = 1 THEN CAST ('' as INT) ELSE 0 END", "null")
testSqlApi("IF(true, CAST ('non-numeric' AS BIGINT), 0)", "null")
}
@@ -177,17 +173,13 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase {
def testTemporalTypeEqualsStringLiteral(): Unit = {
testSqlApi("f15 = '1996-11-10'", "true")
testSqlApi("f15 = '1996-11-11'", "false")
- testSqlApi("f15 = cast(null as string)", "null")
testSqlApi("'1996-11-10' = f15", "true")
testSqlApi("'1996-11-11' = f15", "false")
- testSqlApi("cast(null as string) = f15", "null")
testSqlApi("f21 = '12:34:56'", "true")
testSqlApi("f21 = '13:34:56'", "false")
- testSqlApi("f21 = cast(null as string)", "null")
testSqlApi("'12:34:56' = f21", "true")
testSqlApi("'13:34:56' = f21", "false")
- testSqlApi("cast(null as string) = f21", "null")
testSqlApi("f22 = '1996-11-10 12:34:56'", "true")
testSqlApi("f22 = '1996-11-10 12:34:57'", "false")
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
index 0b8a62d..4db37ea 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
@@ -935,7 +935,7 @@ class CalcITCase extends BatchTestBase {
}
@Test
- def testCast(): Unit = {
+ def testCastInWhere(): Unit = {
checkResult(
"SELECT CAST(a AS VARCHAR(10)) FROM Table3 WHERE CAST(a AS VARCHAR(10)) = '1'",
Seq(row(1)))
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CalcITCase.scala
index e090681..0db5d84 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CalcITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CalcITCase.scala
@@ -342,38 +342,6 @@ class CalcITCase extends BatchTestBase {
}
@Test
- def testCasting() {
- val table = BatchTableEnvUtil.fromElements(tEnv, (1, 0.0, 1L, true))
- .select(
- // * -> String
- '_1.cast(STRING), '_2.cast(STRING), '_3.cast(STRING), '_4.cast(STRING),
- // NUMERIC TYPE -> Boolean
- '_1.cast(BOOLEAN), '_3.cast(BOOLEAN),
- // NUMERIC TYPE -> NUMERIC TYPE
- '_1.cast(DOUBLE), '_2.cast(INT), '_3.cast(SMALLINT),
- // Boolean -> NUMERIC TYPE
- '_4.cast(DOUBLE),
- // identity casting
- '_1.cast(INT), '_2.cast(DOUBLE), '_3.cast(BIGINT), '_4.cast(BOOLEAN))
-
- val results = executeQuery(table)
- val expected = "1,0.0,1,true," + "true,true," +
- "1.0,0,1," + "1.0," + "1,0.0,1,true\n"
- compareResultAsText(results.asJava, expected)
- }
-
- @Test
- def testCastFromString() {
- val table = BatchTableEnvUtil.fromElements(tEnv, ("1", "true", "2.0"))
- .select('_1.cast(TINYINT), '_1.cast(SMALLINT), '_1.cast(INT), '_1.cast(BIGINT),
- '_3.cast(DOUBLE), '_3.cast(FLOAT), '_2.cast(BOOLEAN))
-
- val results = executeQuery(table)
- val expected = "1,1,1,1,2.0,2.0,true\n"
- compareResultAsText(results.asJava, expected)
- }
-
- @Test
def testUserDefinedScalarFunctionWithParameter(): Unit = {
registerFunction("RichFunc2", new RichFunc2)
UserDefinedFunctionTestUtils.setJobParameters(env, Map("string.value" -> "ABC"))
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
index dca4238..54ccb7f 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
@@ -52,27 +52,6 @@ class CalcITCase extends StreamingTestBase {
def usesLegacyRows: LegacyRowResource = LegacyRowResource.INSTANCE
@Test
- def testCastNumericToBooleanInProjection(): Unit ={
- val sqlQuery =
- "SELECT CAST(1 AS BOOLEAN), CAST(0 AS BOOLEAN)"
-
- val outputType = InternalTypeInfo.ofFields(
- new BooleanType(),
- new BooleanType()
- )
-
- val result = tEnv.sqlQuery(sqlQuery).toAppendStream[RowData]
- val sink = new TestingAppendRowDataSink(outputType)
- result.addSink(sink)
- env.execute()
-
- val expected = List(
- "+I(true,false)"
- )
- assertEquals(expected.sorted, sink.getAppendResults.sorted)
- }
-
- @Test
def testCastNumericToBooleanInCondition(): Unit ={
val sqlQuery =
s"""