This is an automated email from the ASF dual-hosted git repository.
snuyanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 5fef51f6008 [FLINK-31300][table] TRY_CAST for constructed types
5fef51f6008 is described below
commit 5fef51f60085306f2acc8c8d630fe08e64004fc3
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Thu Mar 9 23:13:07 2023 +0100
[FLINK-31300][table] TRY_CAST for constructed types
---
.../functions/casting/AbstractCastRule.java | 7 ----
.../table/planner/functions/casting/CastRule.java | 4 ++-
.../planner/functions/CastFunctionMiscITCase.java | 42 +++++++++++++++++++++-
3 files changed, 44 insertions(+), 9 deletions(-)
diff --git
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCastRule.java
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCastRule.java
index a0bcac0dd74..c193139b2f0 100644
---
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCastRule.java
+++
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCastRule.java
@@ -18,8 +18,6 @@
package org.apache.flink.table.planner.functions.casting;
-import org.apache.flink.table.types.logical.LogicalType;
-
/** Base class for all cast rules. */
abstract class AbstractCastRule<IN, OUT> implements CastRule<IN, OUT> {
@@ -33,9 +31,4 @@ abstract class AbstractCastRule<IN, OUT> implements
CastRule<IN, OUT> {
public CastRulePredicate getPredicateDefinition() {
return predicate;
}
-
- @Override
- public boolean canFail(LogicalType inputLogicalType, LogicalType
targetLogicalType) {
- return false;
- }
}
diff --git
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java
index 6b6e036d763..94d3d44ebdb 100644
---
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java
+++
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java
@@ -46,7 +46,9 @@ public interface CastRule<IN, OUT> {
Context context, LogicalType inputLogicalType, LogicalType
targetLogicalType);
/** Returns true if the {@link CastExecutor} can fail at runtime. */
- boolean canFail(LogicalType inputLogicalType, LogicalType
targetLogicalType);
+ default boolean canFail(LogicalType inputLogicalType, LogicalType
targetLogicalType) {
+ return false;
+ }
/** Casting context. */
interface Context {
diff --git
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
index d3c69578c99..ca1e3910b42 100644
---
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
+++
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
@@ -30,6 +30,7 @@ import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.stream.Stream;
+import static org.apache.flink.table.api.DataTypes.ARRAY;
import static org.apache.flink.table.api.DataTypes.BIGINT;
import static org.apache.flink.table.api.DataTypes.BINARY;
import static org.apache.flink.table.api.DataTypes.BOOLEAN;
@@ -268,7 +269,46 @@ class CastFunctionMiscITCase extends
BuiltInFunctionTestBase {
$("f1").tryCast(ROW(TINYINT(), TIME())),
"TRY_CAST(f1 AS ROW(f0 TINYINT, f1 TIME))",
Row.of((byte) 1, LocalTime.of(12, 34, 56, 0)),
- ROW(TINYINT(), TIME()).nullable()));
+ ROW(TINYINT(), TIME()).nullable()),
+ TestSetSpec.forFunction(
+ BuiltInFunctionDefinitions.TRY_CAST, "try cast
from ARRAY to ARRAY")
+ .onFieldsWithData(
+ new String[] {"a"},
+ new Object[] {map(entry(new String[] {"b"},
2))},
+ new String[][] {{"c"}})
+ .andDataTypes(
+ ARRAY(STRING()),
+ ARRAY(MAP(STRING(), STRING())),
+ ARRAY(ARRAY(STRING())))
+ .testResult(
+ $("f0").tryCast(ARRAY(INT())),
+ "TRY_CAST(f0 AS ARRAY<INT>)",
+ null,
+ ARRAY(INT()).nullable())
+ .testResult(
+ $("f1").tryCast(ARRAY(MAP(INT(), INT()))),
+ "TRY_CAST(f1 AS ARRAY<MAP<INT, INT>>)",
+ null,
+ ARRAY(MAP(INT(), INT())).nullable())
+ .testResult(
+ $("f2").tryCast(ARRAY(ARRAY(INT()))),
+ "TRY_CAST(f2 AS ARRAY<ARRAY<INT>>)",
+ null,
+ ARRAY(ARRAY(INT())).nullable()),
+ TestSetSpec.forFunction(
+ BuiltInFunctionDefinitions.TRY_CAST, "try cast
from MAP to MAP")
+ .onFieldsWithData(map(entry("a", 2)), map(entry(1, new
String[] {"a"})))
+ .andDataTypes(MAP(STRING(), INT()), MAP(INT(),
ARRAY(STRING())))
+ .testResult(
+ $("f0").tryCast(MAP(INT(), INT())),
+ "TRY_CAST(f0 AS MAP<INT, INT>)",
+ null,
+ MAP(INT(), INT()).nullable())
+ .testResult(
+ $("f1").tryCast(MAP(INT(), ARRAY(INT()))),
+ "TRY_CAST(f1 AS MAP<INT, ARRAY<INT>>)",
+ null,
+ MAP(INT(), ARRAY(INT())).nullable()));
}
//
--------------------------------------------------------------------------------------------