This is an automated email from the ASF dual-hosted git repository. twalthr pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git
commit d1445f115473efd389a2eadd8ec756cbcdfbde47 Author: slinkydeveloper <francescogu...@gmail.com> AuthorDate: Mon Mar 21 10:47:57 2022 +0100 [FLINK-26770][table-planner] Fix ArrayToArrayCastRule result data structure This closes #19188. --- .../functions/casting/ArrayToArrayCastRule.java | 28 ++------------- .../planner/functions/casting/CastRulesTest.java | 42 ++++++++++++++++++++-- 2 files changed, 42 insertions(+), 28 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java index b67b76a..f3478af 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java @@ -20,8 +20,8 @@ package org.apache.flink.table.planner.functions.casting; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.planner.codegen.CodeGenUtils; import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -109,30 +109,8 @@ class ArrayToArrayCastRule extends AbstractNullAwareCodeGeneratorCastRule<ArrayD private static String arrayElementType(LogicalType t) { if (t.isNullable()) { - return "Object"; + return CodeGenUtils.boxedTypeTermForType(t); } - switch (t.getTypeRoot()) { - case BOOLEAN: - return "boolean"; - case TINYINT: - return "byte"; - case SMALLINT: - return "short"; - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case INTERVAL_YEAR_MONTH: - return "int"; - case BIGINT: - case INTERVAL_DAY_TIME: - return "long"; - case FLOAT: - return "float"; - case DOUBLE: - return "double"; - case DISTINCT_TYPE: - return arrayElementType(((DistinctType) t).getSourceType()); - } - return "Object"; + return CodeGenUtils.primitiveTypeTermForType(t); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java index 1261279..6fdc984 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.base.LocalDateTimeSerializer; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -1258,8 +1259,8 @@ class CastRulesTest { CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().nullable())) .fromCase( ARRAY(INT().nullable()), - new GenericArrayData(new Object[] {1, null, 2}), - new GenericArrayData(new Object[] {1L, null, 2L})), + new GenericArrayData(new Integer[] {1, null, 2}), + new GenericArrayData(new Long[] {1L, null, 2L})), CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().notNull())) .fromCase( ARRAY(INT().notNull()), @@ -1273,7 +1274,42 @@ class CastRulesTest { new GenericArrayData(new Integer[] {1, 2, null}), new GenericArrayData(new Integer[] {3}) }), - NullPointerException.class), + NullPointerException.class) + .fromCase( + ARRAY(ARRAY(INT().nullable())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Long[] {1L, 2L}), + new GenericArrayData(new Long[] {3L}) + })), + CastTestSpecBuilder.testCastTo(ARRAY(ARRAY(DECIMAL(10, 2).notNull()))) + .fromCase( + ARRAY(ARRAY(INT().notNull())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + BigDecimal.ONE, 10, 2), + DecimalData.fromBigDecimal( + new BigDecimal(2), 10, 2) + }), + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + new BigDecimal(3), 10, 2) + }) + })), CastTestSpecBuilder.testCastTo(MAP(DOUBLE().notNull(), DOUBLE().notNull())) .fromCase( MAP(INT().nullable(), INT().nullable()),