From be575dbe2702e8754650b25e6392c7982de0652f Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Thu, 6 Feb 2025 10:05:53 +0100 Subject: [PATCH 01/15] [FLINK-37267][table] Add support for UNNEST WITH ORDINALITY --- .../functions/BuiltInFunctionDefinitions.java | 11 + .../plan/rules/logical/LogicalUnnestRule.java | 24 ++- .../UncollectToTableFunctionScanRule.java | 15 +- .../functions/table/UnnestRowsFunction.java | 138 +++--------- .../table/UnnestRowsFunctionBase.java | 198 ++++++++++++++++++ .../UnnestRowsWithOrdinalityFunction.java | 146 +++++++++++++ 6 files changed, 408 insertions(+), 124 deletions(-) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index 305b6e4c17ee6..616e3a366b1cf 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -477,6 +477,17 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) .internal() .build(); + public static final BuiltInFunctionDefinition INTERNAL_UNNEST_ROWS_WITH_ORDINALITY = + BuiltInFunctionDefinition.newBuilder() + .name("$UNNEST_ROWS_WITH_ORDINALITY$1") + .kind(TABLE) + .inputTypeStrategy(sequence(ANY)) + .outputTypeStrategy(SpecificTypeStrategies.UNUSED) + .runtimeClass( + "org.apache.flink.table.runtime.functions.table.UnnestRowsWithOrdinalityFunction") + .internal() + .build(); + public static final BuiltInFunctionDefinition INTERNAL_HASHCODE = BuiltInFunctionDefinition.newBuilder() .name("$HASHCODE$1") diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java index 6d2eb7a93e6e2..dcc2f938960e7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java @@ -22,7 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; import org.apache.flink.table.planner.utils.ShortcutUtils; -import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction; +import org.apache.flink.table.runtime.functions.table.UnnestRowsFunctionBase; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.shaded.guava32.com.google.common.collect.ImmutableList; @@ -68,12 +68,12 @@ public boolean matches(RelOptRuleCall call) { LogicalFilter logicalFilter = (LogicalFilter) right; RelNode relNode = getRel(logicalFilter.getInput()); if (relNode instanceof Uncollect) { - return !((Uncollect) relNode).withOrdinality; + return true; } else if (relNode instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) relNode; relNode = getRel(logicalProject.getInput()); if (relNode instanceof Uncollect) { - return !((Uncollect) relNode).withOrdinality; + return true; } return false; } @@ -81,13 +81,11 @@ public boolean matches(RelOptRuleCall call) { LogicalProject logicalProject = (LogicalProject) right; RelNode relNode = getRel(logicalProject.getInput()); if (relNode instanceof Uncollect) { - Uncollect uncollect = (Uncollect) relNode; - return !uncollect.withOrdinality; + return true; } return false; } else if (right instanceof Uncollect) { - Uncollect uncollect = (Uncollect) right; - return !uncollect.withOrdinality; + return true; } return false; } @@ -131,16 +129,22 @@ private RelNode convert(RelNode relNode, LogicalCorrelate correlate) { ((Map.Entry) uncollect.getInput().getRowType().getFieldList().get(0)) .getValue(); LogicalType logicalType = FlinkTypeFactory.toLogicalType(relDataType); + BridgingSqlFunction sqlFunction = BridgingSqlFunction.of( - cluster, BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); + cluster, + uncollect.withOrdinality + ? BuiltInFunctionDefinitions + .INTERNAL_UNNEST_ROWS_WITH_ORDINALITY + : BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); RexNode rexCall = cluster.getRexBuilder() .makeCall( typeFactory.createFieldTypeFromLogicalType( toRowType( - UnnestRowsFunction.getUnnestedType( - logicalType))), + UnnestRowsFunctionBase.getUnnestedType( + logicalType, + uncollect.withOrdinality))), sqlFunction, ((LogicalProject) getRel(uncollect.getInput())).getProjects()); return new LogicalTableFunctionScan( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/UncollectToTableFunctionScanRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/UncollectToTableFunctionScanRule.java index 13d49191189e6..5e3d55e13f2c5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/UncollectToTableFunctionScanRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/UncollectToTableFunctionScanRule.java @@ -22,7 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; import org.apache.flink.table.planner.utils.ShortcutUtils; -import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction; +import org.apache.flink.table.runtime.functions.table.UnnestRowsFunctionBase; import org.apache.flink.table.types.logical.LogicalType; import org.apache.calcite.plan.RelOptCluster; @@ -37,7 +37,6 @@ import org.apache.calcite.rel.logical.LogicalValues; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.SqlFunction; import org.immutables.value.Value; import java.util.Collections; @@ -91,14 +90,20 @@ private RelNode convertUncollect(Uncollect uc) { RelDataType relDataType = uc.getInput().getRowType().getFieldList().get(0).getValue(); LogicalType logicalType = FlinkTypeFactory.toLogicalType(relDataType); - SqlFunction sqlFunction = - BridgingSqlFunction.of(cluster, BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); + BridgingSqlFunction sqlFunction = + BridgingSqlFunction.of( + cluster, + uc.withOrdinality + ? BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY + : BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); RexNode rexCall = cluster.getRexBuilder() .makeCall( typeFactory.createFieldTypeFromLogicalType( - toRowType(UnnestRowsFunction.getUnnestedType(logicalType))), + toRowType( + UnnestRowsFunctionBase.getUnnestedType( + logicalType, uc.withOrdinality))), sqlFunction, ((LogicalProject) getRel(uc.getInput())).getProjects()); return new LogicalTableFunctionScan( diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java index 8c6b57efe0959..f0aa6fb76eb78 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java @@ -19,18 +19,11 @@ package org.apache.flink.table.runtime.functions.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.MapData; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.UserDefinedFunction; -import org.apache.flink.table.runtime.functions.BuiltInSpecializedFunction; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; /** @@ -38,150 +31,77 @@ * specialization using a subclass of {@link UnnestTableFunctionBase}. */ @Internal -public class UnnestRowsFunction extends BuiltInSpecializedFunction { +public class UnnestRowsFunction extends UnnestRowsFunctionBase { public UnnestRowsFunction() { - super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); + super(); } @Override - public UserDefinedFunction specialize(SpecializedContext context) { - final LogicalType argType = - context.getCallContext().getArgumentDataTypes().get(0).getLogicalType(); - switch (argType.getTypeRoot()) { - case ARRAY: - final ArrayType arrayType = (ArrayType) argType; - return new CollectionUnnestTableFunction( - context, - arrayType.getElementType(), - ArrayData.createElementGetter(arrayType.getElementType())); - case MULTISET: - final MultisetType multisetType = (MultisetType) argType; - return new CollectionUnnestTableFunction( - context, - multisetType.getElementType(), - ArrayData.createElementGetter(multisetType.getElementType())); - case MAP: - final MapType mapType = (MapType) argType; - return new MapUnnestTableFunction( - context, - RowType.of(false, mapType.getKeyType(), mapType.getValueType()), - ArrayData.createElementGetter(mapType.getKeyType()), - ArrayData.createElementGetter(mapType.getValueType())); - default: - throw new UnsupportedOperationException("Unsupported type for UNNEST: " + argType); - } - } - - public static LogicalType getUnnestedType(LogicalType logicalType) { - switch (logicalType.getTypeRoot()) { - case ARRAY: - return ((ArrayType) logicalType).getElementType(); - case MULTISET: - return ((MultisetType) logicalType).getElementType(); - case MAP: - final MapType mapType = (MapType) logicalType; - return RowType.of(false, mapType.getKeyType(), mapType.getValueType()); - default: - throw new UnsupportedOperationException("Unsupported UNNEST type: " + logicalType); - } + protected UserDefinedFunction createCollectionUnnestFunction( + SpecializedContext context, + LogicalType elementType, + ArrayData.ElementGetter elementGetter) { + return new CollectionUnnestFunction(context, elementType, elementGetter); } - // -------------------------------------------------------------------------------------------- - // Runtime Implementation - // -------------------------------------------------------------------------------------------- - - private abstract static class UnnestTableFunctionBase extends BuiltInTableFunction { - - private final transient DataType outputDataType; - - UnnestTableFunctionBase(SpecializedContext context, LogicalType outputType) { - super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS, context); - // The output type in the context is already wrapped, however, the result of the - // function is not. Therefore, we need a custom output type. - outputDataType = DataTypes.of(outputType).toInternal(); - } - - @Override - public DataType getOutputDataType() { - return outputDataType; - } + @Override + protected UserDefinedFunction createMapUnnestFunction( + SpecializedContext context, + RowType keyValTypes, + ArrayData.ElementGetter keyGetter, + ArrayData.ElementGetter valueGetter) { + return new MapUnnestFunction(context, keyValTypes, keyGetter, valueGetter); } /** Table function that unwraps the elements of a collection (array or multiset). */ - public static final class CollectionUnnestTableFunction extends UnnestTableFunctionBase { + public static final class CollectionUnnestFunction extends UnnestTableFunctionBase { private static final long serialVersionUID = 1L; private final ArrayData.ElementGetter elementGetter; - public CollectionUnnestTableFunction( + public CollectionUnnestFunction( SpecializedContext context, - LogicalType outputType, + LogicalType elementType, ArrayData.ElementGetter elementGetter) { - super(context, outputType); + super(context, elementType); this.elementGetter = elementGetter; } public void eval(ArrayData arrayData) { - if (arrayData == null) { - return; - } - final int size = arrayData.size(); - for (int pos = 0; pos < size; pos++) { - collect(elementGetter.getElementOrNull(arrayData, pos)); - } + evalArrayData(arrayData, elementGetter, (element, position) -> collect(element)); } public void eval(MapData mapData) { - if (mapData == null) { - return; - } - final int size = mapData.size(); - final ArrayData keys = mapData.keyArray(); - final ArrayData values = mapData.valueArray(); - for (int pos = 0; pos < size; pos++) { - final int multiplier = values.getInt(pos); - final Object key = elementGetter.getElementOrNull(keys, pos); - for (int i = 0; i < multiplier; i++) { - collect(key); - } - } + evalMultisetData(mapData, elementGetter, (element, position) -> collect(element)); } } /** Table function that unwraps the elements of a map. */ - public static final class MapUnnestTableFunction extends UnnestTableFunctionBase { + public static final class MapUnnestFunction extends UnnestTableFunctionBase { private static final long serialVersionUID = 1L; private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - public MapUnnestTableFunction( + public MapUnnestFunction( SpecializedContext context, - LogicalType outputType, + LogicalType keyValTypes, ArrayData.ElementGetter keyGetter, ArrayData.ElementGetter valueGetter) { - super(context, outputType); + super(context, keyValTypes); this.keyGetter = keyGetter; this.valueGetter = valueGetter; } public void eval(MapData mapData) { - if (mapData == null) { - return; - } - final int size = mapData.size(); - final ArrayData keyArray = mapData.keyArray(); - final ArrayData valueArray = mapData.valueArray(); - for (int i = 0; i < size; i++) { - collect( - GenericRowData.of( - keyGetter.getElementOrNull(keyArray, i), - valueGetter.getElementOrNull(valueArray, i))); - } + evalMapData( + mapData, + keyGetter, + valueGetter, + (key, value, position) -> collect(GenericRowData.of(key, value))); } } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java new file mode 100644 index 0000000000000..c270ea26bbdc1 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -0,0 +1,198 @@ +/* + * 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.runtime.functions.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.runtime.functions.BuiltInSpecializedFunction; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.RowType; + +/** Base class for flattening ARRAY, MAP, and MULTISET using a table function. */ +@Internal +public abstract class UnnestRowsFunctionBase extends BuiltInSpecializedFunction { + + public UnnestRowsFunctionBase() { + super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); + } + + @Override + public UserDefinedFunction specialize(SpecializedContext context) { + final LogicalType argType = + context.getCallContext().getArgumentDataTypes().get(0).getLogicalType(); + switch (argType.getTypeRoot()) { + case ARRAY: + final ArrayType arrayType = (ArrayType) argType; + return createCollectionUnnestFunction( + context, + arrayType.getElementType(), + ArrayData.createElementGetter(arrayType.getElementType())); + case MULTISET: + final MultisetType multisetType = (MultisetType) argType; + return createCollectionUnnestFunction( + context, + multisetType.getElementType(), + ArrayData.createElementGetter(multisetType.getElementType())); + case MAP: + final MapType mapType = (MapType) argType; + return createMapUnnestFunction( + context, + RowType.of(false, mapType.getKeyType(), mapType.getValueType()), + ArrayData.createElementGetter(mapType.getKeyType()), + ArrayData.createElementGetter(mapType.getValueType())); + default: + throw new UnsupportedOperationException("Unsupported type for UNNEST: " + argType); + } + } + + protected abstract UserDefinedFunction createCollectionUnnestFunction( + SpecializedContext context, + LogicalType elementType, + ArrayData.ElementGetter elementGetter); + + protected abstract UserDefinedFunction createMapUnnestFunction( + SpecializedContext context, + RowType keyValTypes, + ArrayData.ElementGetter keyGetter, + ArrayData.ElementGetter valueGetter); + + public static LogicalType getUnnestedType(LogicalType logicalType, boolean withOrdinality) { + LogicalType baseType; + switch (logicalType.getTypeRoot()) { + case ARRAY: + baseType = ((ArrayType) logicalType).getElementType(); + break; + case MULTISET: + baseType = ((MultisetType) logicalType).getElementType(); + break; + case MAP: + MapType mapType = (MapType) logicalType; + if (withOrdinality) { + return RowType.of( + false, + new LogicalType[] { + mapType.getKeyType(), + mapType.getValueType(), + DataTypes.INT().notNull().getLogicalType() + }, + new String[] {"f0", "f1", "ordinality"}); + } + return RowType.of(false, mapType.getKeyType(), mapType.getValueType()); + default: + throw new UnsupportedOperationException("Unsupported UNNEST type: " + logicalType); + } + + if (withOrdinality) { + return RowType.of( + false, + new LogicalType[] {baseType, DataTypes.INT().notNull().getLogicalType()}, + new String[] {"f0", "ordinality"}); + } + return baseType; + } + + // -------------------------------------------------------------------------------------------- + // Runtime Implementation Base Classes + // -------------------------------------------------------------------------------------------- + + /** Base class for table functions that unwrap collections and maps. */ + protected abstract static class UnnestTableFunctionBase extends BuiltInTableFunction { + private final transient DataType outputDataType; + + UnnestTableFunctionBase(SpecializedContext context, LogicalType elementType) { + super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS, context); + outputDataType = DataTypes.of(elementType).toInternal(); + } + + // The output type in the context is already wrapped, however, the result of the function + // is not. Therefore, we this function has to be implemented with the custom output type. + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + protected void evalArrayData( + ArrayData arrayData, + ArrayData.ElementGetter elementGetter, + UnnestCollector collector) { + if (arrayData == null) { + return; + } + final int size = arrayData.size(); + for (int pos = 0; pos < size; pos++) { + collector.collect(elementGetter.getElementOrNull(arrayData, pos), pos + 1); + } + } + + protected void evalMapData( + MapData mapData, + ArrayData.ElementGetter keyGetter, + ArrayData.ElementGetter valueGetter, + MapUnnestCollector collector) { + if (mapData == null) { + return; + } + final int size = mapData.size(); + final ArrayData keyArray = mapData.keyArray(); + final ArrayData valueArray = mapData.valueArray(); + for (int i = 0; i < size; i++) { + collector.collect( + keyGetter.getElementOrNull(keyArray, i), + valueGetter.getElementOrNull(valueArray, i), + i + 1); + } + } + + protected void evalMultisetData( + MapData mapData, ArrayData.ElementGetter elementGetter, UnnestCollector collector) { + if (mapData == null) { + return; + } + final int size = mapData.size(); + final ArrayData keys = mapData.keyArray(); + final ArrayData values = mapData.valueArray(); + int ordinal = 1; + for (int pos = 0; pos < size; pos++) { + final int multiplier = values.getInt(pos); + final Object key = elementGetter.getElementOrNull(keys, pos); + for (int i = 0; i < multiplier; i++) { + collector.collect(key, ordinal++); + } + } + } + + @FunctionalInterface + protected interface UnnestCollector { + void collect(Object element, int position); + } + + @FunctionalInterface + protected interface MapUnnestCollector { + void collect(Object key, Object value, int position); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java new file mode 100644 index 0000000000000..ac8c988f67ddf --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -0,0 +1,146 @@ +/* + * 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.runtime.functions.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +/** + * Flattens ARRAY, MAP, and MULTISET using a table function and adds one extra column with the + * position of the element. It does this by another level of specialization using a subclass of + * {@link UnnestTableFunctionBase}. + */ +@Internal +public class UnnestRowsWithOrdinalityFunction extends UnnestRowsFunctionBase { + + public UnnestRowsWithOrdinalityFunction() { + super(); + } + + @Override + protected UserDefinedFunction createCollectionUnnestFunction( + SpecializedContext context, + LogicalType elementType, + ArrayData.ElementGetter elementGetter) { + return new CollectionUnnestWithOrdinalityFunction(context, elementType, elementGetter); + } + + @Override + protected UserDefinedFunction createMapUnnestFunction( + SpecializedContext context, + RowType keyValTypes, + ArrayData.ElementGetter keyGetter, + ArrayData.ElementGetter valueGetter) { + return new MapUnnestWithOrdinalityFunction(context, keyValTypes, keyGetter, valueGetter); + } + + /** + * Table function that unwraps the elements of a collection (array or multiset) with ordinality. + */ + public static final class CollectionUnnestWithOrdinalityFunction + extends UnnestTableFunctionBase { + + private static final long serialVersionUID = 1L; + + private final ArrayData.ElementGetter elementGetter; + private final transient DataType outputDataType; + + public CollectionUnnestWithOrdinalityFunction( + SpecializedContext context, + LogicalType elementType, + ArrayData.ElementGetter elementGetter) { + super(context, elementType); + this.elementGetter = elementGetter; + + outputDataType = + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.of(elementType).notNull()), + DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) + .toInternal(); + } + + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + public void eval(ArrayData arrayData) { + evalArrayData( + arrayData, + elementGetter, + (element, position) -> collect(GenericRowData.of(element, position))); + } + + public void eval(MapData mapData) { + evalMultisetData( + mapData, + elementGetter, + (element, position) -> collect(GenericRowData.of(element, position))); + } + } + + /** Table function that unwraps the elements of a map with ordinality. */ + public static final class MapUnnestWithOrdinalityFunction extends UnnestTableFunctionBase { + + private static final long serialVersionUID = 1L; + + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private final transient DataType outputDataType; + + public MapUnnestWithOrdinalityFunction( + SpecializedContext context, + LogicalType keyValTypes, + ArrayData.ElementGetter keyGetter, + ArrayData.ElementGetter valueGetter) { + super(context, keyValTypes); + this.keyGetter = keyGetter; + this.valueGetter = valueGetter; + + RowType rowType = (RowType) keyValTypes; + outputDataType = + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.of(rowType.getTypeAt(0))), + DataTypes.FIELD("f1", DataTypes.of(rowType.getTypeAt(1))), + DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) + .toInternal(); + } + + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + public void eval(MapData mapData) { + evalMapData( + mapData, + keyGetter, + valueGetter, + (key, value, position) -> collect(GenericRowData.of(key, value, position))); + } + } +} From 32d5725f02aa28800ac4869d84649c56a879f032 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Thu, 6 Feb 2025 10:07:45 +0100 Subject: [PATCH 02/15] [FLINK-37267][table] Add stream, batch and plan tests for withordinality --- .../planner/plan/batch/sql/UnnestTest.xml | 198 ++++++++++++ .../rules/logical/LogicalUnnestRuleTest.xml | 212 ++++++++++++ .../planner/plan/stream/sql/UnnestTest.xml | 193 +++++++++++ .../planner/plan/common/UnnestTestBase.scala | 73 +++++ .../runtime/batch/sql/UnnestITCase.scala | 80 +++++ .../runtime/stream/sql/UnnestITCase.scala | 302 ++++++++++++++++++ 6 files changed, 1058 insertions(+) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index 46cdbd1e6dd43..06fc4f4532ecc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -352,6 +352,204 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) Calc(select=[a, _1 AS b1, _2 AS b2]) +- Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER], condition=[AND(>=($0, 12), <>($1, _UTF-16LE'Hello'))]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 10 AND ordinality < 3 +]]> + + + ($2, 10), <($3, 3))]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(number=[$0], ordinality=[$1]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 10), <($1, 3))]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index edd58ca7be47a..65379de52be58 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -371,6 +371,218 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b1=[$0], b2=[$1]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 10 AND ordinality < 3 +]]> + + + ($2, 10), <($3, 3))]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(number=[$0], ordinality=[$1]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($2, 10), <($3, 3))]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(number=[$0], ordinality=[$1]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index dcfcc0c4e3b40..f482b3081b9e9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -364,6 +364,199 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) Calc(select=[a, _1 AS b1, _2 AS b2]) +- Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER], condition=[AND(>=($0, 12), <>($1, _UTF-16LE'Hello'))]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 10 AND ordinality < 3 +]]> + + + ($2, 10), <($3, 3))]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(number=[$0], ordinality=[$1]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 10), <($1, 3))]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala index a16b91c7b1225..faf873fced163 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala @@ -145,6 +145,79 @@ abstract class UnnestTestBase(withExecPlan: Boolean) extends TableTestBase { verifyPlan(sqlQuery) } + @Test + def testUnnestWithOrdinalityArray(): Unit = { + util.addTableSource[(Int, Array[Int])]("MyTable", 'a, 'b) + verifyPlan( + "SELECT a, number, ordinality FROM MyTable CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality)") + } + + @Test + def testUnnestWithOrdinalityAndValues(): Unit = { + verifyPlan("SELECT val, pos FROM UNNEST(ARRAY[1,2,3]) WITH ORDINALITY AS t(val, pos)") + } + + @Test + def testUnnestWithOrdinalityArrayOfArray(): Unit = { + util.addTableSource[(Int, Array[Array[Int]])]("MyTable", 'id, 'nested_array) + val sqlQuery = + """ + |SELECT id, array_val, array_pos, elem, element_pos + |FROM MyTable + |CROSS JOIN UNNEST(nested_array) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(array_val) WITH ORDINALITY AS B(elem, element_pos) + |""".stripMargin + verifyPlan(sqlQuery) + } + + @Test + def testUnnestWithOrdinalityMultiset(): Unit = { + util.addDataStream[(Int, String, String)]("MyTable", 'a, 'b, 'c) + val sqlQuery = + """ + |WITH T AS (SELECT a, COLLECT(c) as words FROM MyTable GROUP BY a) + |SELECT a, word, pos + |FROM T CROSS JOIN UNNEST(words) WITH ORDINALITY AS A(word, pos) + |""".stripMargin + verifyPlan(sqlQuery) + } + + @Test + def testUnnestWithOrdinalityMap(): Unit = { + util.addTableSource( + "MyTable", + Array[AbstractDataType[_]]( + DataTypes.INT(), + DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())), + Array("id", "map_data")) + verifyPlan( + "SELECT id, k, v, pos FROM MyTable CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos)") + } + + @Test + def testUnnestWithOrdinalityWithFilter(): Unit = { + util.addTableSource[(Int, Array[Int])]("MyTable", 'a, 'b) + verifyPlan(""" + |SELECT a, number, ordinality + |FROM MyTable + |CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |WHERE number > 10 AND ordinality < 3 + |""".stripMargin) + } + + @Test + def testUnnestWithOrdinalityInSubquery(): Unit = { + util.addTableSource[(Int, Array[Int])]("MyTable", 'a, 'b) + verifyPlan(""" + |SELECT * FROM ( + | SELECT a, number, ordinality + | FROM MyTable + | CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |) subquery + |WHERE ordinality = 1 + |""".stripMargin) + } + def verifyPlan(sql: String): Unit = { if (withExecPlan) { util.verifyExecPlan(sql) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala index 970e2db1141b2..71d6bcbe83250 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala @@ -315,4 +315,84 @@ class UnnestITCase extends BatchTestBase { Seq(row('a', 1), row('a', 2), row('a', 3))) } + @Test + def testUnnestPrimitiveArrayFromTableWithOrdinality(): Unit = { + val data = List( + row(1, Array(12, 45), Array(Array(12, 45))), + row(2, Array(41, 5), Array(Array(18), Array(87))), + row(3, Array(18, 42), Array(Array(1), Array(45))) + ) + registerCollection( + "T", + data, + new RowTypeInfo( + Types.INT, + Types.PRIMITIVE_ARRAY(Types.INT), + Types.OBJECT_ARRAY(Types.PRIMITIVE_ARRAY(Types.INT))), + "a, b, c") + + checkResult( + "SELECT a, b, s, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A (s, o)", + Seq( + row(1, Array(12, 45), 12, 1), + row(1, Array(12, 45), 45, 2), + row(2, Array(41, 5), 41, 1), + row(2, Array(41, 5), 5, 2), + row(3, Array(18, 42), 18, 1), + row(3, Array(18, 42), 42, 2)) + ) + } + + @Test + def testUnnestArrayOfArrayFromTableWithOrdinality(): Unit = { + val data = List( + row(1, Array(12, 45), Array(Array(12, 45))), + row(2, Array(41, 5), Array(Array(18), Array(87))), + row(3, Array(18, 42), Array(Array(1), Array(45))) + ) + registerCollection( + "T", + data, + new RowTypeInfo( + Types.INT, + Types.PRIMITIVE_ARRAY(Types.INT), + Types.OBJECT_ARRAY(Types.PRIMITIVE_ARRAY(Types.INT))), + "a, b, c") + + checkResult( + "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY AS A (s, o)", + Seq( + row(1, Array(12, 45), 1), + row(2, Array(18), 1), + row(2, Array(87), 2), + row(3, Array(1), 1), + row(3, Array(45), 2)) + ) + } + + @Test + def testCrossWithUnnestForMapWithOrdinality(): Unit = { + val data = List( + row(1, 11L, Map("a" -> "10", "b" -> "11").asJava), + row(2, 22L, Map("c" -> "20").asJava), + row(3, 33L, Map("d" -> "30", "e" -> "31").asJava) + ) + + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.LONG, Types.MAP(Types.STRING, Types.STRING)), + "a, b, c") + + checkResult( + "SELECT a, b, v, o FROM T CROSS JOIN UNNEST(c) WITH ORDINALITY as f (k, v, o)", + Seq( + row(1, 11L, "10", 1), + row(1, 11L, "11", 2), + row(2, 22L, "20", 1), + row(3, 33L, "30", 1), + row(3, 33L, "31", 2)) + ) + } + } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 1a0d18c76ded1..8e7e9e84c6d6b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -379,4 +379,306 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo val expected = List("a,1", "a,2", "a,3") assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) } + + @TestTemplate + def testUnnestWithOrdinalityWithValuesStream(): Unit = { + val sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3]) WITH ORDINALITY" + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List("a,1,1", "a,2,2", "a,3,3") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestArrayWithOrdinality(): Unit = { + val data = List( + (1, Array(12, 45)), + (2, Array(41, 5)), + (3, Array(18, 42)) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) + tEnv.createTemporaryView("T", t) + + val sqlQuery = """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List("1,12,1", "1,45,2", "2,41,1", "2,5,2", "3,18,1", "3,42,2") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestArrayOfArrayWithOrdinality(): Unit = { + val data = List( + (1, Array(Array(1, 2), Array(3))), + (2, Array(Array(4, 5), Array(6, 7))), + (3, Array(Array(8))) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'nested_array) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |SELECT id, array_val, array_pos, `element`, element_pos + |FROM T + |CROSS JOIN UNNEST(nested_array) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(array_val) WITH ORDINALITY AS B(`element`, element_pos) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List( + "1,[1, 2],1,1,1", + "1,[1, 2],1,2,2", + "1,[3],2,3,1", + "2,[4, 5],1,4,1", + "2,[4, 5],1,5,2", + "2,[6, 7],2,6,1", + "2,[6, 7],2,7,2", + "3,[8],1,8,1") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestMultisetWithOrdinality(): Unit = { + val data = List( + (1, 1, "Hi"), + (1, 2, "Hello"), + (2, 2, "World"), + (3, 3, "Hello world") + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |WITH T1 AS (SELECT a, COLLECT(c) as words FROM T GROUP BY a) + |SELECT a, word, pos + |FROM T1 CROSS JOIN UNNEST(words) WITH ORDINALITY AS A(word, pos) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] + val sink = new TestingRetractSink + result.addSink(sink) + env.execute() + + val expected = List( + "1,Hi,1", + "1,Hello,2", + "2,World,1", + "3,Hello world,1" + ) + assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestMapWithOrdinality(): Unit = { + val data = List( + Row.of( + Int.box(1), { + val map = new java.util.HashMap[String, String]() + map.put("a", "10") + map.put("b", "11") + map + }), + Row.of( + Int.box(2), { + val map = new java.util.HashMap[String, String]() + map.put("c", "20") + map.put("d", "21") + map + }) + ) + + implicit val typeInfo = Types.ROW( + Array("id", "map_data"), + Array[TypeInformation[_]](Types.INT, Types.MAP(Types.STRING, Types.STRING)) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'map_data) + tEnv.createTemporaryView("T", t) + + val sqlQuery = """ + |SELECT id, k, v, pos + |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val resultsWithoutordinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) + val expected = List("1,a,10", "1,b,11", "2,c,20", "2,d,21") + + assertThat(resultsWithoutordinality.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestWithOrdinalityForChainOfArraysAndMaps(): Unit = { + val data = List( + Row.of( + Int.box(1), + Array("a", "b"), { + val map = new java.util.HashMap[String, String]() + map.put("x", "10") + map.put("y", "20") + map + }), + Row.of( + Int.box(2), + Array("c", "d"), { + val map = new java.util.HashMap[String, String]() + map.put("z", "30") + map.put("w", "40") + map + }) + ) + + implicit val typeInfo = Types.ROW( + Array("id", "array_data", "map_data"), + Array[TypeInformation[_]]( + Types.INT, + Types.OBJECT_ARRAY(Types.STRING), + Types.MAP(Types.STRING, Types.STRING)) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'array_data, 'map_data) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |SELECT id, array_val, array_pos, map_key, map_val, map_pos + |FROM T + |CROSS JOIN UNNEST(array_data) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS B(map_key, map_val, map_pos) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val resultsWithoutOrdinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) + val expected = List( + "1,a,1,x,10", + "1,a,1,y,20", + "1,b,2,x,10", + "1,b,2,y,20", + "2,c,1,z,30", + "2,c,1,w,40", + "2,d,2,z,30", + "2,d,2,w,40" + ) + + assertThat(resultsWithoutOrdinality.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestWithOrdinalityForEmptyArray(): Unit = { + val data = List( + (1, Array[Int]()) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) + tEnv.createTemporaryView("T", t) + + val sqlQuery = """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List() + assertThat(sink.getAppendResults.sorted).isEqualTo(expected) + } + + @TestTemplate + def testUnnestWithOrdinalityForMapWithNullValues(): Unit = { + val data = List( + Row.of( + Int.box(1), { + val map = new java.util.HashMap[String, String]() + map.put("a", "10") + map.put("b", null) + map + }), + Row.of( + Int.box(2), { + val map = new java.util.HashMap[String, String]() + map.put("c", "20") + map.put("d", null) + map + }) + ) + + implicit val typeInfo = Types.ROW( + Array("id", "map_data"), + Array[TypeInformation[_]](Types.INT, Types.MAP(Types.STRING, Types.STRING)) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'map_data) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |SELECT id, k, v, pos + |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val resultsWithoutordinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) + val expected = List("1,a,10", "1,b,null", "2,c,20", "2,d,null") + assertThat(resultsWithoutordinality.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestArrayWithMixedTypesAndOrdinality(): Unit = { + val data = List( + (1, Array(10, "20")), + (2, Array(30, "40")) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS m(number, ordinality) + |""".stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List("1,10,1", "1,20,2", "2,30,1", "2,40,2") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + + /* Utility for maps to assert that ordinality is within range and remove it from output. + * Necessary since maps are not ordered */ + def assertAndRemoveOrdinality(results: List[String], maxOrdinality: Int): List[String] = { + results.foreach { + result => + val columns = result.split(",") + val ordinality = columns.last.toInt + assert( + ordinality >= 1 && ordinality <= maxOrdinality, + s"Ordinality $ordinality out of range") + } + + results.map(_.split(",").dropRight(1).mkString(",")) + } } From 9972d3c099a88de59b9d8ecf86ce4a8a4db41392 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Thu, 6 Feb 2025 10:06:30 +0100 Subject: [PATCH 03/15] [FLINK-37267][docs] Revamp outdated unnest documentation --- .../docs/dev/table/sql/queries/joins.md | 47 +++++++++++++++++-- .../plan/rules/logical/LogicalUnnestRule.java | 14 ++---- .../table/UnnestRowsFunctionBase.java | 8 ++-- 3 files changed, 50 insertions(+), 19 deletions(-) diff --git a/docs/content/docs/dev/table/sql/queries/joins.md b/docs/content/docs/dev/table/sql/queries/joins.md index 97f672ab8ac8a..37a0f1082481c 100644 --- a/docs/content/docs/dev/table/sql/queries/joins.md +++ b/docs/content/docs/dev/table/sql/queries/joins.md @@ -326,16 +326,55 @@ FROM Orders AS o In the example above, the Orders table is enriched with data from the Customers table which resides in a MySQL database. The `FOR SYSTEM_TIME AS OF` clause with the subsequent processing time attribute ensures that each row of the `Orders` table is joined with those Customers rows that match the join predicate at the point in time when the `Orders` row is processed by the join operator. It also prevents that the join result is updated when a joined `Customer` row is updated in the future. The lookup join also requires a mandatory equality join predicate, in the example above `o.customer_id = c.id`. -Array Expansion +Array, Multiset and Map Expansion -------------- -Returns a new row for each element in the given array. Unnesting `WITH ORDINALITY` is not yet supported. +Unnest returns a new row for each element in the given array, multiset or map. Supports both `CROSS JOIN` and `LEFT JOIN`. +```sql +-- Returns a new row for each element in a constant array +SELECT * FROM (VALUES('order_1')), UNNEST(ARRAY["shirt", "pants", "hat"]) + +id product_name +======= ============ +order_1 shirt +order_1 pants +order_1 hat + +-- Returns a new row for each element in the array +-- assuming a Orders table with an array column `product_names` +SELECT order_id, product_name +FROM Orders + CROSS JOIN UNNEST(product_names) AS t(product_name) +``` + +Unnesting `WITH ORDINALITY` is also supported. + ```sql -SELECT order_id, tag -FROM Orders CROSS JOIN UNNEST(tags) AS t (tag) +-- Returns a new row for each element in a constant array and its position in the array +SELECT * +FROM (VALUES('order_1')) + CROSS JOIN UNNEST(ARRAY["shirt", "pants", "hat"]) + WITH ORDINALITY AS t(product_name, index) + +id product_name index +======= ============ ===== +order_1 shirt 1 +order_1 pants 2 +order_1 hat 3 + +-- Returns a new row for each element and its position in the array +-- assuming a Orders table with an array column `product_names` +SELECT order_id, product_name, product_index +FROM Orders + CROSS JOIN UNNEST(product_names) + WITH ORDINALITY AS t(product_name, product_index) ``` +A unnest with ordinality will return each element and the position of the element in the data structure, 1-indexed. +The order of the elements for arrays is guaranteed. Since maps and multisets are unordered, the order of the elements is not guaranteed. +Currently, WITH ORDINALITY only supports cross joins but not left joins. + Table Function -------------- diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java index dcc2f938960e7..c7ae8f8a3cf94 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java @@ -72,21 +72,13 @@ public boolean matches(RelOptRuleCall call) { } else if (relNode instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) relNode; relNode = getRel(logicalProject.getInput()); - if (relNode instanceof Uncollect) { - return true; - } - return false; + return relNode instanceof Uncollect; } } else if (right instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) right; RelNode relNode = getRel(logicalProject.getInput()); - if (relNode instanceof Uncollect) { - return true; - } - return false; - } else if (right instanceof Uncollect) { - return true; - } + return relNode instanceof Uncollect; + } else return right instanceof Uncollect; return false; } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java index c270ea26bbdc1..c39014f6644d9 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -159,11 +159,11 @@ protected void evalMapData( final int size = mapData.size(); final ArrayData keyArray = mapData.keyArray(); final ArrayData valueArray = mapData.valueArray(); - for (int i = 0; i < size; i++) { + for (int pos = 0; pos < size; pos++) { collector.collect( - keyGetter.getElementOrNull(keyArray, i), - valueGetter.getElementOrNull(valueArray, i), - i + 1); + keyGetter.getElementOrNull(keyArray, pos), + valueGetter.getElementOrNull(valueArray, pos), + pos + 1); } } From 57137d2f0ca44d096981e8e1fb780cad553a63dc Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Fri, 7 Feb 2025 17:10:35 +0100 Subject: [PATCH 04/15] [FLINK-37267][table] Support with ordinality for array/multiset of rows as calcite implements it. Also added several tests --- .../planner/plan/batch/sql/UnnestTest.xml | 45 ++++ .../rules/logical/LogicalUnnestRuleTest.xml | 52 ++++ .../planner/plan/stream/sql/UnnestTest.xml | 45 ++++ .../planner/plan/common/UnnestTestBase.scala | 18 +- .../runtime/batch/sql/UnnestITCase.scala | 234 +++++++++++++++--- .../runtime/stream/sql/UnnestITCase.scala | 141 ++++++++++- .../table/UnnestRowsFunctionBase.java | 43 ++-- .../UnnestRowsWithOrdinalityFunction.java | 71 ++++-- 8 files changed, 569 insertions(+), 80 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index 06fc4f4532ecc..63f245d7b3770 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -429,6 +429,51 @@ Calc(select=[id, f0 AS array_val, ordinality AS array_pos, f00 AS elem, ordinali +- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val))], select=[id,nested_array,f0,ordinality,f00,ordinality0], rowType=[RecordType(INTEGER id, INTEGER ARRAY ARRAY nested_array, INTEGER ARRAY f0, INTEGER ordinality, INTEGER f00, INTEGER ordinality0)], joinType=[INNER]) +- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array))], select=[id,nested_array,f0,ordinality], rowType=[RecordType(INTEGER id, INTEGER ARRAY ARRAY nested_array, INTEGER ARRAY f0, INTEGER ordinality)], joinType=[INNER]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[id, nested_array]) +]]> + + + + + 13]]> + + + ($2, 13)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(s=[$0], t=[$1], o=[$2]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 13)]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + 1]]> + + + ($2, 1)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 1)]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index 65379de52be58..fa097c67b9648 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -457,6 +457,58 @@ LogicalProject(id=[$0], array_val=[$2], array_pos=[$3], elem=[$4], element_pos=[ : +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array)], rowType=[RecordType:peek_no_expand(INTEGER ARRAY f0, INTEGER ordinality)]) +- LogicalProject(elem=[$0], element_pos=[$1]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) +]]> + + + + + 13]]> + + + ($2, 13)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(s=[$0], t=[$1], o=[$2]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($2, 13)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(s=[$0], t=[$1], o=[$2]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, VARCHAR(2147483647) f1, INTEGER ordinality)]) +]]> + + + + + 1]]> + + + ($2, 1)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($2, 1)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, VARCHAR(2147483647) f1, INTEGER ordinality)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index f482b3081b9e9..bb3de22851a70 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -441,6 +441,51 @@ Calc(select=[id, f0 AS array_val, ordinality AS array_pos, f00 AS elem, ordinali +- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val))], select=[id,nested_array,f0,ordinality,f00,ordinality0], rowType=[RecordType(INTEGER id, INTEGER ARRAY ARRAY nested_array, INTEGER ARRAY f0, INTEGER ordinality, INTEGER f00, INTEGER ordinality0)], joinType=[INNER]) +- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array))], select=[id,nested_array,f0,ordinality], rowType=[RecordType(INTEGER id, INTEGER ARRAY ARRAY nested_array, INTEGER ARRAY f0, INTEGER ordinality)], joinType=[INNER]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[id, nested_array]) +]]> + + + + + 13]]> + + + ($2, 13)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalProject(s=[$0], t=[$1], o=[$2]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 13)]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) +]]> + + + + + 1]]> + + + ($2, 1)]) + +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- Uncollect(withOrdinality=[true]) + +- LogicalProject(b=[$cor0.b]) + +- LogicalValues(tuples=[[{ 0 }]]) +]]> + + + ($0, 1)]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala index faf873fced163..3385013d84382 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala @@ -145,6 +145,11 @@ abstract class UnnestTestBase(withExecPlan: Boolean) extends TableTestBase { verifyPlan(sqlQuery) } + @Test + def testUnnestWithOrdinalityAndValues(): Unit = { + verifyPlan("SELECT val, pos FROM UNNEST(ARRAY[1,2,3]) WITH ORDINALITY AS t(val, pos)") + } + @Test def testUnnestWithOrdinalityArray(): Unit = { util.addTableSource[(Int, Array[Int])]("MyTable", 'a, 'b) @@ -153,8 +158,17 @@ abstract class UnnestTestBase(withExecPlan: Boolean) extends TableTestBase { } @Test - def testUnnestWithOrdinalityAndValues(): Unit = { - verifyPlan("SELECT val, pos FROM UNNEST(ARRAY[1,2,3]) WITH ORDINALITY AS t(val, pos)") + def testUnnestWithOrdinalityArrayOfRowsWithoutAlias(): Unit = { + util.addTableSource[(Int, Array[(Int, String)])]("MyTable", 'a, 'b) + verifyPlan( + "SELECT a, b, A._1, A._2, A.`ORDINALITY` FROM MyTable, UNNEST(MyTable.b) WITH ORDINALITY AS A where A._1 > 1") + } + + @Test + def testUnnestWithOrdinalityArrayOfRowsFromTableWithFilter(): Unit = { + util.addTableSource[(Int, Array[(Int, String)])]("MyTable", 'a, 'b) + verifyPlan( + "SELECT a, b, s, t, o FROM MyTable, UNNEST(MyTable.b) WITH ORDINALITY AS A (s, t, o) WHERE s > 13") } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala index 71d6bcbe83250..4ea7b3977be0d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala @@ -316,82 +316,234 @@ class UnnestITCase extends BatchTestBase { } @Test - def testUnnestPrimitiveArrayFromTableWithOrdinality(): Unit = { + def testUnnestWithOrdinalityWithValuesStream(): Unit = { + checkResult( + "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3]) WITH ORDINALITY", + Seq(row('a', 1, 1), row('a', 2, 2), row('a', 3, 3)) + ) + } + + @Test + def testUnnestArrayWithOrdinality(): Unit = { val data = List( - row(1, Array(12, 45), Array(Array(12, 45))), - row(2, Array(41, 5), Array(Array(18), Array(87))), - row(3, Array(18, 42), Array(Array(1), Array(45))) + row(1, Array(12, 45)), + row(2, Array(41, 5)), + row(3, Array(18, 42)) ) registerCollection( "T", data, - new RowTypeInfo( - Types.INT, - Types.PRIMITIVE_ARRAY(Types.INT), - Types.OBJECT_ARRAY(Types.PRIMITIVE_ARRAY(Types.INT))), + new RowTypeInfo(Types.INT, Types.PRIMITIVE_ARRAY(Types.INT)), + "a, b") + + checkResult( + """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin, + Seq(row(1, 12, 1), row(1, 45, 2), row(2, 41, 1), row(2, 5, 2), row(3, 18, 1), row(3, 42, 2)) + ) + } + + @Test + def testUnnestFromTableWithOrdinality(): Unit = { + val data = List( + row(1, 1L, Array("Hi", "w")), + row(2, 2L, Array("Hello", "k")), + row(3, 2L, Array("Hello world", "x")) + ) + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.LONG, Types.OBJECT_ARRAY(Types.STRING)), "a, b, c") checkResult( - "SELECT a, b, s, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A (s, o)", + "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY as A (s, o)", Seq( - row(1, Array(12, 45), 12, 1), - row(1, Array(12, 45), 45, 2), - row(2, Array(41, 5), 41, 1), - row(2, Array(41, 5), 5, 2), - row(3, Array(18, 42), 18, 1), - row(3, Array(18, 42), 42, 2)) + row(1, "Hi", 1), + row(1, "w", 2), + row(2, "Hello", 1), + row(2, "k", 2), + row(3, "Hello world", 1), + row(3, "x", 2)) ) } @Test - def testUnnestArrayOfArrayFromTableWithOrdinality(): Unit = { + def testUnnestArrayOfArrayWithOrdinality(): Unit = { val data = List( - row(1, Array(12, 45), Array(Array(12, 45))), - row(2, Array(41, 5), Array(Array(18), Array(87))), - row(3, Array(18, 42), Array(Array(1), Array(45))) + row(1, Array(Array(1, 2), Array(3))), + row(2, Array(Array(4, 5), Array(6, 7))), + row(3, Array(Array(8))) + ) + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.OBJECT_ARRAY(Types.PRIMITIVE_ARRAY(Types.INT))), + "id, nested_array") + + checkResult( + """ + |SELECT id, array_val, array_pos, `element`, element_pos + |FROM T + |CROSS JOIN UNNEST(nested_array) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(array_val) WITH ORDINALITY AS B(`element`, element_pos) + |""".stripMargin, + Seq( + row(1, Array(1, 2), 1, 1, 1), + row(1, Array(1, 2), 1, 2, 2), + row(1, Array(3), 2, 3, 1), + row(2, Array(4, 5), 1, 4, 1), + row(2, Array(4, 5), 1, 5, 2), + row(2, Array(6, 7), 2, 6, 1), + row(2, Array(6, 7), 2, 7, 2), + row(3, Array(8), 1, 8, 1) + ) + ) + } + + @Test + def testUnnestMultisetWithOrdinality(): Unit = { + val data = List( + row(1, 1, "Hi"), + row(1, 2, "Hello"), + row(2, 2, "World"), + row(3, 3, "Hello world") + ) + registerCollection("T", data, new RowTypeInfo(Types.INT, Types.INT, Types.STRING), "a, b, c") + + checkResult( + """ + |WITH T1 AS (SELECT a, COLLECT(c) as words FROM T GROUP BY a) + |SELECT a, word, pos + |FROM T1 CROSS JOIN UNNEST(words) WITH ORDINALITY AS A(word, pos) + |""".stripMargin, + Seq(row(1, "Hi", 1), row(1, "Hello", 2), row(2, "World", 1), row(3, "Hello world", 1)) + ) + } + + @Test + def testUnnestMapWithOrdinality(): Unit = { + val data = List( + row(1, Map("a" -> "10", "b" -> "11").asJava), + row(2, Map("c" -> "20", "d" -> "21").asJava) + ) + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.MAP(Types.STRING, Types.STRING)), + "id, map_data") + + checkResult( + """ + |SELECT id, k, v + |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) + |""".stripMargin, + Seq(row(1, "a", "10"), row(1, "b", "11"), row(2, "c", "20"), row(2, "d", "21")) + ) + } + + @Test + def testUnnestWithOrdinalityForChainOfArraysAndMaps(): Unit = { + val data = List( + row(1, Array("a", "b"), Map("x" -> "10", "y" -> "20").asJava), + row(2, Array("c", "d"), Map("z" -> "30", "w" -> "40").asJava) ) registerCollection( "T", data, new RowTypeInfo( Types.INT, - Types.PRIMITIVE_ARRAY(Types.INT), - Types.OBJECT_ARRAY(Types.PRIMITIVE_ARRAY(Types.INT))), - "a, b, c") + Types.OBJECT_ARRAY(Types.STRING), + Types.MAP(Types.STRING, Types.STRING)), + "id, array_data, map_data") checkResult( - "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY AS A (s, o)", + """ + |SELECT id, array_val, array_pos, map_key, map_val, map_pos + |FROM T + |CROSS JOIN UNNEST(array_data) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS B(map_key, map_val, map_pos) + |""".stripMargin, Seq( - row(1, Array(12, 45), 1), - row(2, Array(18), 1), - row(2, Array(87), 2), - row(3, Array(1), 1), - row(3, Array(45), 2)) + row(1, "a", 1, "x", "10", 1), + row(1, "a", 1, "y", "20", 2), + row(1, "b", 2, "x", "10", 1), + row(1, "b", 2, "y", "20", 2), + row(2, "c", 1, "z", "30", 1), + row(2, "c", 1, "w", "40", 2), + row(2, "d", 2, "z", "30", 1), + row(2, "d", 2, "w", "40", 2) + ) ) } @Test - def testCrossWithUnnestForMapWithOrdinality(): Unit = { - val data = List( - row(1, 11L, Map("a" -> "10", "b" -> "11").asJava), - row(2, 22L, Map("c" -> "20").asJava), - row(3, 33L, Map("d" -> "30", "e" -> "31").asJava) + def testUnnestWithOrdinalityForEmptyArray(): Unit = { + val data = List(row(1, Array[Int]())) + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.PRIMITIVE_ARRAY(Types.INT)), + "a, b") + + checkResult( + """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin, + Seq() ) + } + @Test + def testUnnestArrayOfRowsFromTableWithOrdinality(): Unit = { + val data = List( + row(2, Array(row(20, "41.6"), row(14, "45.2136"))), + row(3, Array(row(18, "42.6"))) + ) registerCollection( "T", data, - new RowTypeInfo(Types.INT, Types.LONG, Types.MAP(Types.STRING, Types.STRING)), - "a, b, c") + new RowTypeInfo(Types.INT, Types.OBJECT_ARRAY(new RowTypeInfo(Types.INT, Types.STRING))), + "a, b") checkResult( - "SELECT a, b, v, o FROM T CROSS JOIN UNNEST(c) WITH ORDINALITY as f (k, v, o)", + "SELECT a, b, s, t, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A(s, t, o)", Seq( - row(1, 11L, "10", 1), - row(1, 11L, "11", 2), - row(2, 22L, "20", 1), - row(3, 33L, "30", 1), - row(3, 33L, "31", 2)) + row(2, Array(row(20, "41.6"), row(14, "45.2136")), 20, "41.6", 1), + row(2, Array(row(20, "41.6"), row(14, "45.2136")), 14, "45.2136", 2), + row(3, Array(row(18, "42.6")), 18, "42.6", 1) + ) + ) + } + + @Test + def testUnnestArrayOfRowsWithNestedFilterWithOrdinality(): Unit = { + val data = List( + row(1, Array(row(12, "45.6"), row(12, "45.612"))), + row(2, Array(row(13, "41.6"), row(14, "45.2136"))), + row(3, Array(row(18, "42.6"))) + ) + registerCollection( + "T", + data, + new RowTypeInfo(Types.INT, Types.OBJECT_ARRAY(new RowTypeInfo(Types.INT, Types.STRING))), + "a, b") + + checkResult( + """ + |SELECT * FROM ( + | SELECT a, b1, b2, ord FROM + | (SELECT a, b FROM T) T + | CROSS JOIN + | UNNEST(T.b) WITH ORDINALITY as S(b1, b2, ord) + | WHERE S.b1 >= 12 + | ) tmp + |WHERE b2 <> '42.6' AND ord <> 2 + |""".stripMargin, + Seq(row(1, 12, "45.6", 1), row(2, 13, "41.6", 1)) ) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 8e7e9e84c6d6b..d8df4473d0ed8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.core.testutils.EachCallbackWrapper import org.apache.flink.table.api._ +import org.apache.flink.table.api.DataTypes import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.connector.ChangelogMode import org.apache.flink.table.legacy.api.Types @@ -326,7 +327,7 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo } @TestTemplate - def testUnnestWithNestedFilter(): Unit = { + def testUnnestArrayOfRowsWithNestedFilter(): Unit = { val data = List( (1, Array((12, "45.6"), (12, "45.612"))), (2, Array((13, "41.6"), (14, "45.2136"))), @@ -415,6 +416,27 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) } + @TestTemplate + def testUnnestFromTableWithOrdinality(): Unit = { + val data = List( + (1, 1L, Array("Hi", "w")), + (2, 2L, Array("Hello", "k")), + (3, 2L, Array("Hello world", "x")) + ) + + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) + tEnv.createTemporaryView("T", t) + + val sqlQuery = "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY as A (s, o)" + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List("1,Hi,1", "1,w,2", "2,Hello,1", "2,k,2", "3,Hello world,1", "3,x,2") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + @TestTemplate def testUnnestArrayOfArrayWithOrdinality(): Unit = { val data = List( @@ -645,28 +667,114 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo } @TestTemplate - def testUnnestArrayWithMixedTypesAndOrdinality(): Unit = { + def testUnnestArrayOfRowsFromTableWithOrdinality(): Unit = { val data = List( - (1, Array(10, "20")), - (2, Array(30, "40")) + (2, Array((20, "41.6"), (14, "45.2136"))), + (3, Array((18, "42.6"))) ) val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) tEnv.createTemporaryView("T", t) + val sqlQuery = + "SELECT a, b, s, t, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A(s, t, o)" + val result = tEnv.sqlQuery(sqlQuery).toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = List( + "2,[20,41.6, 14,45.2136],20,41.6,1", + "2,[20,41.6, 14,45.2136],14,45.2136,2", + "3,[18,42.6],18,42.6,1") + assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testUnnestArrayOfRowsWithNestedFilterWithOrdinality(): Unit = { + val data = List( + (1, Array((12, "45.6"), (12, "45.612"))), + (2, Array((13, "41.6"), (14, "45.2136"))), + (3, Array((18, "42.6"))) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) + tEnv.createTemporaryView("MyTable", t) + val sqlQuery = """ - |SELECT a, number, ordinality - |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS m(number, ordinality) - |""".stripMargin + |SELECT * FROM ( + | SELECT a, b1, b2, ord FROM + | (SELECT a, b FROM MyTable) T + | CROSS JOIN + | UNNEST(T.b) WITH ORDINALITY as S(b1, b2, ord) + | WHERE S.b1 >= 12 + | ) tmp + |WHERE b2 <> '42.6' AND ord <> 2 + """.stripMargin + val result = tEnv.sqlQuery(sqlQuery).toDataStream val sink = new TestingAppendSink result.addSink(sink) env.execute() - val expected = List("1,10,1", "1,20,2", "2,30,1", "2,40,2") + val expected = List("1,12,45.6,1", "2,13,41.6,1") assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) } + @TestTemplate + def testUnnestMultiSetFromCollectResultWithOrdinality(): Unit = { + val data = List( + (1, 1, (12, "45.6")), + (2, 2, (12, "45.612")), + (3, 2, (13, "41.6")), + (4, 3, (14, "45.2136")), + (5, 3, (18, "42.6"))) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |WITH T1 AS (SELECT b, COLLECT(c) as `set` FROM T GROUP BY b) + |SELECT b, id, point, o FROM T1 + |CROSS JOIN UNNEST(T1.`set`) WITH ORDINALITY AS A(id, point, o) WHERE b < 3 + """.stripMargin + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] + val sink = new TestingRetractSink + result.addSink(sink) + env.execute() + + val expected = List("1,12,45.6,1", "2,12,45.612,1", "2,13,41.6,2") + assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + } + + @TestTemplate + def testTumbleWindowAggregateWithCollectUnnestWithOrdinality(): Unit = { + val data = TestData.tupleData3.map { case (i, l, s) => (l, i, s) } + val stream = failingDataSource(data) + .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset[(Long, Int, String)](0L)) + val t = stream.toTable(tEnv, 'b, 'a, 'c, 'rowtime.rowtime) + tEnv.createTemporaryView("T", t) + + val sqlQuery = + """ + |WITH T1 AS (SELECT b, COLLECT(b) as `set` + | FROM T + | GROUP BY b, TUMBLE(rowtime, INTERVAL '3' SECOND) + |) + |SELECT b, s, o FROM T1, UNNEST(T1.`set`) WITH ORDINALITY AS A(s, o) where b < 3 + """.stripMargin + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] + val sink = new TestingRetractSink + result.addSink(sink).setParallelism(1) + env.execute() + + val expected = List( + "1,1,1", + "2,2,1", + "2,2,2" + ) + assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + } + /* Utility for maps to assert that ordinality is within range and remove it from output. * Necessary since maps are not ordered */ def assertAndRemoveOrdinality(results: List[String], maxOrdinality: Int): List[String] = { @@ -681,4 +789,21 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo results.map(_.split(",").dropRight(1).mkString(",")) } + + @TestTemplate + def testUnnestWithOrdinalityAliasColumnNames(): Unit = { + val sqlQuery = + """ + |SELECT * FROM (VALUES('a')) AS T(x) + |CROSS JOIN UNNEST(ARRAY[1,2,3]) WITH ORDINALITY AS U(y, ord) + """.stripMargin + + val result = tEnv.sqlQuery(sqlQuery) + + val fieldNames = result.getResolvedSchema.getColumnNames.toList + + val expectedFieldNames = List("x", "y", "ord") + assertThat(fieldNames).isEqualTo(expectedFieldNames) + } + } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java index c39014f6644d9..cad6e3ea0c22e 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -81,38 +81,51 @@ protected abstract UserDefinedFunction createMapUnnestFunction( ArrayData.ElementGetter valueGetter); public static LogicalType getUnnestedType(LogicalType logicalType, boolean withOrdinality) { - LogicalType baseType; + LogicalType elementType; switch (logicalType.getTypeRoot()) { case ARRAY: - baseType = ((ArrayType) logicalType).getElementType(); + elementType = ((ArrayType) logicalType).getElementType(); break; case MULTISET: - baseType = ((MultisetType) logicalType).getElementType(); + elementType = ((MultisetType) logicalType).getElementType(); break; case MAP: MapType mapType = (MapType) logicalType; - if (withOrdinality) { - return RowType.of( - false, - new LogicalType[] { - mapType.getKeyType(), - mapType.getValueType(), - DataTypes.INT().notNull().getLogicalType() - }, - new String[] {"f0", "f1", "ordinality"}); - } - return RowType.of(false, mapType.getKeyType(), mapType.getValueType()); + elementType = RowType.of(false, mapType.getKeyType(), mapType.getValueType()); + break; default: throw new UnsupportedOperationException("Unsupported UNNEST type: " + logicalType); } if (withOrdinality) { + return wrapWithOrdinality(elementType); + } + return elementType; + } + + public static LogicalType wrapWithOrdinality(LogicalType baseType) { + // If baseType is already a ROW, extract its fields and add an ordinality field + if (baseType instanceof RowType) { + RowType rowType = (RowType) baseType; + int fieldCount = rowType.getFieldCount(); + LogicalType[] types = new LogicalType[fieldCount + 1]; + String[] names = new String[types.length]; + + for (int i = 0; i < fieldCount; i++) { + types[i] = rowType.getTypeAt(i); + names[i] = "f" + i; + } + + types[fieldCount] = DataTypes.INT().notNull().getLogicalType(); + names[fieldCount] = "ordinality"; + return RowType.of(false, types, names); + } else { + // For non-row types, wrap in a row with f0 and ordinality return RowType.of( false, new LogicalType[] {baseType, DataTypes.INT().notNull().getLogicalType()}, new String[] {"f0", "ordinality"}); } - return baseType; } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java index ac8c988f67ddf..54f3d8cb6a7d2 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -23,11 +23,15 @@ import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import java.util.ArrayList; +import java.util.List; + /** * Flattens ARRAY, MAP, and MULTISET using a table function and adds one extra column with the * position of the element. It does this by another level of specialization using a subclass of @@ -62,11 +66,11 @@ protected UserDefinedFunction createMapUnnestFunction( */ public static final class CollectionUnnestWithOrdinalityFunction extends UnnestTableFunctionBase { - private static final long serialVersionUID = 1L; private final ArrayData.ElementGetter elementGetter; private final transient DataType outputDataType; + private RowData.FieldGetter[] fieldGetters = null; public CollectionUnnestWithOrdinalityFunction( SpecializedContext context, @@ -75,11 +79,13 @@ public CollectionUnnestWithOrdinalityFunction( super(context, elementType); this.elementGetter = elementGetter; - outputDataType = - DataTypes.ROW( - DataTypes.FIELD("f0", DataTypes.of(elementType).notNull()), - DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) - .toInternal(); + if (elementType instanceof RowType) { + RowType rowType = (RowType) elementType; + this.fieldGetters = createFieldGetters(rowType); + this.outputDataType = createRowTypeOutputDataType(rowType); + } else { + this.outputDataType = createSimpleOutputDataType(elementType); + } } @Override @@ -88,17 +94,54 @@ public DataType getOutputDataType() { } public void eval(ArrayData arrayData) { - evalArrayData( - arrayData, - elementGetter, - (element, position) -> collect(GenericRowData.of(element, position))); + evalArrayData(arrayData, elementGetter, this::collectWithOrdinality); } public void eval(MapData mapData) { - evalMultisetData( - mapData, - elementGetter, - (element, position) -> collect(GenericRowData.of(element, position))); + evalMultisetData(mapData, elementGetter, this::collectWithOrdinality); + } + + private void collectWithOrdinality(Object element, int position) { + if (element instanceof RowData) { + RowData innerRow = (RowData) element; + int arity = innerRow.getArity(); + GenericRowData outRow = new GenericRowData(arity + 1); + + for (int i = 0; i < arity; i++) { + outRow.setField(i, fieldGetters[i].getFieldOrNull(innerRow)); + } + + outRow.setField(arity, position); + collect(outRow); + } else { + collect(GenericRowData.of(element, position)); + } + } + + private RowData.FieldGetter[] createFieldGetters(RowType rowType) { + int fieldCount = rowType.getFieldCount(); + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldCount]; + for (int i = 0; i < fieldCount; i++) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + return fieldGetters; + } + + private DataType createRowTypeOutputDataType(RowType rowType) { + int fieldCount = rowType.getFieldCount(); + List fields = new ArrayList<>(); + for (int i = 0; i < fieldCount; i++) { + fields.add(DataTypes.FIELD("f" + i, DataTypes.of(rowType.getTypeAt(i)))); + } + fields.add(DataTypes.FIELD("ordinality", DataTypes.INT().notNull())); + return DataTypes.ROW(fields).toInternal(); + } + + private DataType createSimpleOutputDataType(LogicalType elementType) { + return DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.of(elementType)), + DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) + .toInternal(); } } From ef1df4c77b29ec6249380794ac0caaf7f6fd68b4 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sat, 8 Feb 2025 13:26:00 +0100 Subject: [PATCH 05/15] [FLINK-37267][table] Add test for map of rows with ordinality --- .../plan/rules/logical/LogicalUnnestRule.java | 4 +- .../runtime/batch/sql/UnnestITCase.scala | 45 +++++++- .../runtime/stream/sql/UnnestITCase.scala | 106 +++++++++++++----- .../UnnestRowsWithOrdinalityFunction.java | 4 + 4 files changed, 132 insertions(+), 27 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java index c7ae8f8a3cf94..fcebd54654d34 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java @@ -78,7 +78,9 @@ public boolean matches(RelOptRuleCall call) { LogicalProject logicalProject = (LogicalProject) right; RelNode relNode = getRel(logicalProject.getInput()); return relNode instanceof Uncollect; - } else return right instanceof Uncollect; + } else { + return right instanceof Uncollect; + } return false; } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala index 4ea7b3977be0d..f4fdbe6928dee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala @@ -444,6 +444,50 @@ class UnnestITCase extends BatchTestBase { ) } + @Test + def testUnnestForMapOfRowsWithOrdinality(): Unit = { + val data = List( + row( + 1, { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("a", "a"), Row.of(10: Integer)) + map.put(Row.of("b", "b"), Row.of(11: Integer)) + map + }), + row( + 2, { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("c", "c"), Row.of(20: Integer)) + map + }), + row( + 3, { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("d", "d"), Row.of(30: Integer)) + map.put(Row.of("e", "e"), Row.of(31: Integer)) + map + }) + ) + + registerCollection( + "T", + data, + new RowTypeInfo( + Types.INT, + Types.MAP(Types.ROW(Types.STRING, Types.STRING), Types.ROW(Types.INT()))), + "a, b") + + checkResult( + "SELECT a, k, v, o FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY as f (k, v, o)", + Seq( + row(1, row("a", "a"), row(10), 1), + row(1, row("b", "b"), row(11), 2), + row(2, row("c", "c"), row(20), 1), + row(3, row("d", "d"), row(30), 1), + row(3, row("e", "e"), row(31), 2)) + ) + } + @Test def testUnnestWithOrdinalityForChainOfArraysAndMaps(): Unit = { val data = List( @@ -546,5 +590,4 @@ class UnnestITCase extends BatchTestBase { Seq(row(1, 12, "45.6", 1), row(2, 13, "41.6", 1)) ) } - } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index d8df4473d0ed8..5d2da729bb776 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -544,6 +544,62 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo assertThat(resultsWithoutordinality.sorted).isEqualTo(expected.sorted) } + def testUnnestForMapOfRowsWitOrdinality(): Unit = { + val data = List( + Row.of( + Int.box(1), { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("a", "a"), Row.of(10: Integer)) + map.put(Row.of("b", "b"), Row.of(11: Integer)) + map + }), + Row.of( + Int.box(2), { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("c", "c"), Row.of(20: Integer)) + map + }), + Row.of( + Int.box(3), { + val map = new java.util.HashMap[Row, Row]() + map.put(Row.of("d", "d"), Row.of(30: Integer)) + map.put(Row.of("e", "e"), Row.of(31: Integer)) + map + }) + ) + + implicit val typeInfo = Types.ROW( + Array("a", "b"), + Array[TypeInformation[_]]( + Types.INT, + Types.MAP(Types.ROW(Types.STRING, Types.STRING), Types.ROW(Types.INT()))) + ) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) + tEnv.createTemporaryView("T", t) + + val sqlQuery = "SELECT a, k, v, o FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY as f (k, v, o)" + val result = tEnv.sqlQuery(sqlQuery) + TestSinkUtil.addValuesSink( + tEnv, + "MySink", + List("a", "k", "v", "o"), + List( + DataTypes.INT, + DataTypes.ROW(DataTypes.STRING(), DataTypes.STRING()), + DataTypes.ROW(DataTypes.INT()), + DataTypes.INT.notNull()), + ChangelogMode.all() + ) + result.executeInsert("MySink").await() + + val expected = + List("1,a,a,10", "1,b,b,11", "2,c,c,20", "3,d,d,30", "3,e,e,31") + val resultWithoutOrd = assertAndRemoveOrdinality( + TestValuesTableFactory.getResultsAsStrings("MySink").sorted.toList, + 2) + assertThat(resultWithoutOrd).isEqualTo(expected.sorted) + } + @TestTemplate def testUnnestWithOrdinalityForChainOfArraysAndMaps(): Unit = { val data = List( @@ -721,21 +777,21 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo } @TestTemplate - def testUnnestMultiSetFromCollectResultWithOrdinality(): Unit = { + def testUnnestMultiSetOfRowsFromCollectResultWithOrdinality(): Unit = { val data = List( - (1, 1, (12, "45.6")), - (2, 2, (12, "45.612")), - (3, 2, (13, "41.6")), - (4, 3, (14, "45.2136")), - (5, 3, (18, "42.6"))) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) + (1, (12, "45.6")), + (2, (12, "45.612")), + (2, (13, "41.6")), + (3, (14, "45.2136")), + (3, (18, "42.6"))) + val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) tEnv.createTemporaryView("T", t) val sqlQuery = """ - |WITH T1 AS (SELECT b, COLLECT(c) as `set` FROM T GROUP BY b) - |SELECT b, id, point, o FROM T1 - |CROSS JOIN UNNEST(T1.`set`) WITH ORDINALITY AS A(id, point, o) WHERE b < 3 + |WITH T1 AS (SELECT a, COLLECT(b) as `set` FROM T GROUP BY a) + |SELECT a, id, point, o FROM T1 + |CROSS JOIN UNNEST(T1.`set`) WITH ORDINALITY AS A(id, point, o) WHERE a < 3 """.stripMargin val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] val sink = new TestingRetractSink @@ -775,21 +831,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) } - /* Utility for maps to assert that ordinality is within range and remove it from output. - * Necessary since maps are not ordered */ - def assertAndRemoveOrdinality(results: List[String], maxOrdinality: Int): List[String] = { - results.foreach { - result => - val columns = result.split(",") - val ordinality = columns.last.toInt - assert( - ordinality >= 1 && ordinality <= maxOrdinality, - s"Ordinality $ordinality out of range") - } - - results.map(_.split(",").dropRight(1).mkString(",")) - } - @TestTemplate def testUnnestWithOrdinalityAliasColumnNames(): Unit = { val sqlQuery = @@ -806,4 +847,19 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo assertThat(fieldNames).isEqualTo(expectedFieldNames) } + /* Utility for maps to assert that ordinality is within range and remove it from output. + * Necessary since maps are not ordered */ + def assertAndRemoveOrdinality(results: List[String], maxOrdinality: Int): List[String] = { + results.foreach { + result => + val columns = result.split(",") + val ordinality = columns.last.toInt + assert( + ordinality >= 1 && ordinality <= maxOrdinality, + s"Ordinality $ordinality out of range") + } + + results.map(_.split(",").dropRight(1).mkString(",")) + } + } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java index 54f3d8cb6a7d2..e9db9e9792fff 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -80,6 +80,10 @@ public CollectionUnnestWithOrdinalityFunction( this.elementGetter = elementGetter; if (elementType instanceof RowType) { + /* When unnesting a collection, according to Calcite's implementation, + row(a,b) unnests to a row(a, b, ordinality) and not to (row(a,b), ordinality). + That means, if we are unnesting a row, we need field getters + to be able to extract all field values */ RowType rowType = (RowType) elementType; this.fieldGetters = createFieldGetters(rowType); this.outputDataType = createRowTypeOutputDataType(rowType); From ad7826b92751409328fef496444a58a600b467e9 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sun, 9 Feb 2025 20:56:29 +0100 Subject: [PATCH 06/15] [FLINK-37267][table] Simplify logic for output type with ordinality --- .../UnnestRowsWithOrdinalityFunction.java | 52 ++----------------- 1 file changed, 4 insertions(+), 48 deletions(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java index e9db9e9792fff..0b8841ec93b1f 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -19,19 +19,14 @@ package org.apache.flink.table.runtime.functions.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.MapData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.functions.UserDefinedFunction; -import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; -import java.util.ArrayList; -import java.util.List; - /** * Flattens ARRAY, MAP, and MULTISET using a table function and adds one extra column with the * position of the element. It does this by another level of specialization using a subclass of @@ -49,7 +44,8 @@ protected UserDefinedFunction createCollectionUnnestFunction( SpecializedContext context, LogicalType elementType, ArrayData.ElementGetter elementGetter) { - return new CollectionUnnestWithOrdinalityFunction(context, elementType, elementGetter); + return new CollectionUnnestWithOrdinalityFunction( + context, wrapWithOrdinality(elementType), elementGetter); } @Override @@ -58,7 +54,8 @@ protected UserDefinedFunction createMapUnnestFunction( RowType keyValTypes, ArrayData.ElementGetter keyGetter, ArrayData.ElementGetter valueGetter) { - return new MapUnnestWithOrdinalityFunction(context, keyValTypes, keyGetter, valueGetter); + return new MapUnnestWithOrdinalityFunction( + context, wrapWithOrdinality(keyValTypes), keyGetter, valueGetter); } /** @@ -69,7 +66,6 @@ public static final class CollectionUnnestWithOrdinalityFunction private static final long serialVersionUID = 1L; private final ArrayData.ElementGetter elementGetter; - private final transient DataType outputDataType; private RowData.FieldGetter[] fieldGetters = null; public CollectionUnnestWithOrdinalityFunction( @@ -86,17 +82,9 @@ public CollectionUnnestWithOrdinalityFunction( to be able to extract all field values */ RowType rowType = (RowType) elementType; this.fieldGetters = createFieldGetters(rowType); - this.outputDataType = createRowTypeOutputDataType(rowType); - } else { - this.outputDataType = createSimpleOutputDataType(elementType); } } - @Override - public DataType getOutputDataType() { - return outputDataType; - } - public void eval(ArrayData arrayData) { evalArrayData(arrayData, elementGetter, this::collectWithOrdinality); } @@ -130,23 +118,6 @@ private RowData.FieldGetter[] createFieldGetters(RowType rowType) { } return fieldGetters; } - - private DataType createRowTypeOutputDataType(RowType rowType) { - int fieldCount = rowType.getFieldCount(); - List fields = new ArrayList<>(); - for (int i = 0; i < fieldCount; i++) { - fields.add(DataTypes.FIELD("f" + i, DataTypes.of(rowType.getTypeAt(i)))); - } - fields.add(DataTypes.FIELD("ordinality", DataTypes.INT().notNull())); - return DataTypes.ROW(fields).toInternal(); - } - - private DataType createSimpleOutputDataType(LogicalType elementType) { - return DataTypes.ROW( - DataTypes.FIELD("f0", DataTypes.of(elementType)), - DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) - .toInternal(); - } } /** Table function that unwraps the elements of a map with ordinality. */ @@ -157,8 +128,6 @@ public static final class MapUnnestWithOrdinalityFunction extends UnnestTableFun private final ArrayData.ElementGetter keyGetter; private final ArrayData.ElementGetter valueGetter; - private final transient DataType outputDataType; - public MapUnnestWithOrdinalityFunction( SpecializedContext context, LogicalType keyValTypes, @@ -167,19 +136,6 @@ public MapUnnestWithOrdinalityFunction( super(context, keyValTypes); this.keyGetter = keyGetter; this.valueGetter = valueGetter; - - RowType rowType = (RowType) keyValTypes; - outputDataType = - DataTypes.ROW( - DataTypes.FIELD("f0", DataTypes.of(rowType.getTypeAt(0))), - DataTypes.FIELD("f1", DataTypes.of(rowType.getTypeAt(1))), - DataTypes.FIELD("ordinality", DataTypes.INT().notNull())) - .toInternal(); - } - - @Override - public DataType getOutputDataType() { - return outputDataType; } public void eval(MapData mapData) { From e179c09f3cc980b126bf2a6b4a0b9477518401a6 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Fri, 14 Feb 2025 17:11:32 +0100 Subject: [PATCH 07/15] [FLINK-37267][docs] Complement docs with multiset and map example + typo --- .../docs/dev/table/sql/queries/joins.md | 55 +++++++++++++++++-- 1 file changed, 49 insertions(+), 6 deletions(-) diff --git a/docs/content/docs/dev/table/sql/queries/joins.md b/docs/content/docs/dev/table/sql/queries/joins.md index 37a0f1082481c..7a89da0d35b3e 100644 --- a/docs/content/docs/dev/table/sql/queries/joins.md +++ b/docs/content/docs/dev/table/sql/queries/joins.md @@ -332,7 +332,7 @@ Array, Multiset and Map Expansion Unnest returns a new row for each element in the given array, multiset or map. Supports both `CROSS JOIN` and `LEFT JOIN`. ```sql -- Returns a new row for each element in a constant array -SELECT * FROM (VALUES('order_1')), UNNEST(ARRAY["shirt", "pants", "hat"]) +SELECT * FROM (VALUES('order_1')), UNNEST(ARRAY['shirt', 'pants', 'hat']) id product_name ======= ============ @@ -347,14 +347,14 @@ FROM Orders CROSS JOIN UNNEST(product_names) AS t(product_name) ``` -Unnesting `WITH ORDINALITY` is also supported. +Unnesting `WITH ORDINALITY` is also supported. Currently, `WITH ORDINALITY` only supports `CROSS JOIN` but not `LEFT JOIN`. ```sql -- Returns a new row for each element in a constant array and its position in the array SELECT * -FROM (VALUES('order_1')) - CROSS JOIN UNNEST(ARRAY["shirt", "pants", "hat"]) +FROM (VALUES ('order_1'), ('order_2')) + CROSS JOIN UNNEST(ARRAY['shirt', 'pants', 'hat']) WITH ORDINALITY AS t(product_name, index) id product_name index @@ -362,6 +362,9 @@ id product_name index order_1 shirt 1 order_1 pants 2 order_1 hat 3 +order_2 shirt 1 +order_2 pants 2 +order_2 hat 3 -- Returns a new row for each element and its position in the array -- assuming a Orders table with an array column `product_names` @@ -371,9 +374,49 @@ FROM Orders WITH ORDINALITY AS t(product_name, product_index) ``` -A unnest with ordinality will return each element and the position of the element in the data structure, 1-indexed. +An unnest with ordinality will return each element and the position of the element in the data structure, 1-indexed. The order of the elements for arrays is guaranteed. Since maps and multisets are unordered, the order of the elements is not guaranteed. -Currently, WITH ORDINALITY only supports cross joins but not left joins. + +```sql +-- Returns a new row each key/value pair in the map. +SELECT * +FROM + (VALUES('order_1')) + CROSS JOIN UNNEST(MAP['shirt', 2, 'pants', 1, 'hat', 1]) WITH ORDINALITY + +id product_name amount index +======= ============ ===== ===== +order_1 shirt 2 1 +order_1 pants 1 2 +order_1 hat 1 3 + +-- Returns a new row for each instance of a element in a multiset +-- If an element has been seen twice (multiplicity is 2), it will be returned twice +WITH ProductMultiset AS + (SELECT COLLECT(product_name) AS product_multiset + FROM ( + VALUES ('shirt'), ('pants'), ('hat'), ('shirt'), ('hat') + ) AS t(product_name)) -- produces { 'shirt': 2, 'pants': 1, 'hat': 2 } +SELECT id, product_name, ordinality +FROM + (VALUES ('order_1'), ('order_2')) AS t(id), + ProductMultiset + CROSS JOIN UNNEST(product_multiset) WITH + ORDINALITY AS u(product_name, ordinality); + +id product_name index +======= ============ ===== +order_1 shirt 1 +order_1 shirt 2 +order_1 pants 3 +order_1 hat 4 +order_1 hat 5 +order_2 shirt 1 +order_2 shirt 2 +order_2 pants 3 +order_2 hat 4 +order_1 hat 5 +``` Table Function -------------- From cda44cfae9e81214c22939224adac97864ca9f17 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sat, 15 Feb 2025 12:53:59 +0100 Subject: [PATCH 08/15] [FLINK-37267][table] Pass withOrdinality to base class constructors --- .../functions/table/UnnestRowsFunction.java | 6 +++--- .../functions/table/UnnestRowsFunctionBase.java | 16 ++++++++++++---- .../table/UnnestRowsWithOrdinalityFunction.java | 6 +++--- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java index f0aa6fb76eb78..cece2e1e34afe 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunction.java @@ -34,7 +34,7 @@ public class UnnestRowsFunction extends UnnestRowsFunctionBase { public UnnestRowsFunction() { - super(); + super(false); } @Override @@ -65,7 +65,7 @@ public CollectionUnnestFunction( SpecializedContext context, LogicalType elementType, ArrayData.ElementGetter elementGetter) { - super(context, elementType); + super(context, elementType, false); this.elementGetter = elementGetter; } @@ -91,7 +91,7 @@ public MapUnnestFunction( LogicalType keyValTypes, ArrayData.ElementGetter keyGetter, ArrayData.ElementGetter valueGetter) { - super(context, keyValTypes); + super(context, keyValTypes, false); this.keyGetter = keyGetter; this.valueGetter = valueGetter; } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java index cad6e3ea0c22e..8d1aed4a119cb 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -36,8 +36,11 @@ @Internal public abstract class UnnestRowsFunctionBase extends BuiltInSpecializedFunction { - public UnnestRowsFunctionBase() { - super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); + public UnnestRowsFunctionBase(boolean withOrdinality) { + super( + withOrdinality + ? BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY + : BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS); } @Override @@ -136,8 +139,13 @@ public static LogicalType wrapWithOrdinality(LogicalType baseType) { protected abstract static class UnnestTableFunctionBase extends BuiltInTableFunction { private final transient DataType outputDataType; - UnnestTableFunctionBase(SpecializedContext context, LogicalType elementType) { - super(BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS, context); + UnnestTableFunctionBase( + SpecializedContext context, LogicalType elementType, boolean withOrdinality) { + super( + withOrdinality + ? BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY + : BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS, + context); outputDataType = DataTypes.of(elementType).toInternal(); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java index 0b8841ec93b1f..fa6276347bc29 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -36,7 +36,7 @@ public class UnnestRowsWithOrdinalityFunction extends UnnestRowsFunctionBase { public UnnestRowsWithOrdinalityFunction() { - super(); + super(true); } @Override @@ -72,7 +72,7 @@ public CollectionUnnestWithOrdinalityFunction( SpecializedContext context, LogicalType elementType, ArrayData.ElementGetter elementGetter) { - super(context, elementType); + super(context, elementType, true); this.elementGetter = elementGetter; if (elementType instanceof RowType) { @@ -133,7 +133,7 @@ public MapUnnestWithOrdinalityFunction( LogicalType keyValTypes, ArrayData.ElementGetter keyGetter, ArrayData.ElementGetter valueGetter) { - super(context, keyValTypes); + super(context, keyValTypes, true); this.keyGetter = keyGetter; this.valueGetter = valueGetter; } From 7f522cfdfe1817dc1c4a0162153ec5e93dd0ec5a Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sat, 15 Feb 2025 12:55:13 +0100 Subject: [PATCH 09/15] [FLINK-37267][table] Refactor getFields with new RowType for naming --- .../table/UnnestRowsFunctionBase.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java index 8d1aed4a119cb..a4b73fad12674 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -32,6 +32,9 @@ import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; +import java.util.stream.Collectors; +import java.util.stream.Stream; + /** Base class for flattening ARRAY, MAP, and MULTISET using a table function. */ @Internal public abstract class UnnestRowsFunctionBase extends BuiltInSpecializedFunction { @@ -110,18 +113,15 @@ public static LogicalType wrapWithOrdinality(LogicalType baseType) { // If baseType is already a ROW, extract its fields and add an ordinality field if (baseType instanceof RowType) { RowType rowType = (RowType) baseType; - int fieldCount = rowType.getFieldCount(); - LogicalType[] types = new LogicalType[fieldCount + 1]; - String[] names = new String[types.length]; - - for (int i = 0; i < fieldCount; i++) { - types[i] = rowType.getTypeAt(i); - names[i] = "f" + i; - } - - types[fieldCount] = DataTypes.INT().notNull().getLogicalType(); - names[fieldCount] = "ordinality"; - return RowType.of(false, types, names); + return new RowType( + false, + Stream.concat( + rowType.getFields().stream(), + Stream.of( + new RowType.RowField( + "ordinality", + DataTypes.INT().notNull().getLogicalType()))) + .collect(Collectors.toList())); } else { // For non-row types, wrap in a row with f0 and ordinality return RowType.of( From e54bf99dd04c4f8139eeda71472746bb523d6081 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sat, 15 Feb 2025 14:59:32 +0100 Subject: [PATCH 10/15] [FLINK-37267][table] Refactor UnnestITCase to use parameterized calls --- .../runtime/stream/sql/UnnestITCase.scala | 805 ++++++++---------- 1 file changed, 377 insertions(+), 428 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 5d2da729bb776..ba8eb7480377e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -20,11 +20,9 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.core.testutils.EachCallbackWrapper import org.apache.flink.table.api._ -import org.apache.flink.table.api.DataTypes import org.apache.flink.table.api.bridge.scala._ -import org.apache.flink.table.connector.ChangelogMode +import org.apache.flink.table.expressions.Expression import org.apache.flink.table.legacy.api.Types -import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset @@ -44,6 +42,35 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo @RegisterExtension private val _: EachCallbackWrapper[LegacyRowExtension] = new EachCallbackWrapper[LegacyRowExtension](new LegacyRowExtension) + private def assertUnnest[T]( + testData: List[T], + typeInfo: TypeInformation[T], + sqlQuery: String, + expectedResults: List[String], + isRetract: Boolean, + parallelism: Int, + fieldNames: Expression*): Unit = { + val t = StreamingEnvUtil.fromCollection(env, testData)(typeInfo).toTable(tEnv, fieldNames: _*) + tEnv.createTemporaryView("T", t) + + val result = tEnv.sqlQuery(sqlQuery) + if (isRetract) { + val sink = new TestingRetractSink + if (parallelism > 0) { + result.toRetractStream[Row].addSink(sink).setParallelism(parallelism) + } else { + result.toRetractStream[Row].addSink(sink) + } + env.execute() + assertThat(sink.getRetractResults.sorted).isEqualTo(expectedResults.sorted) + } else { + val sink = new TestingAppendSink + result.toDataStream.addSink(sink) + env.execute() + assertThat(sink.getAppendResults.sorted).isEqualTo(expectedResults.sorted) + } + } + @TestTemplate def testUnnestPrimitiveArrayFromTable(): Unit = { val data = List( @@ -51,23 +78,24 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array(41, 5), Array(Array(18), Array(87))), (3, Array(18, 42), Array(Array(1), Array(45))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, b, s FROM T, UNNEST(T.b) AS A (s)" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - val expected = List( - "1,[12, 45],12", - "1,[12, 45],45", - "2,[41, 5],41", - "2,[41, 5],5", - "3,[18, 42],18", - "3,[18, 42],42") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[Int], Array[Array[Int]])], + sqlQuery = "SELECT a, b, s FROM T, UNNEST(T.b) AS A (s)", + expectedResults = List( + "1,[12, 45],12", + "1,[12, 45],45", + "2,[41, 5],41", + "2,[41, 5],5", + "3,[18, 42],18", + "3,[18, 42],42"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c + ) } @TestTemplate @@ -77,17 +105,17 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array(41, 5), Array(Array(18), Array(87))), (3, Array(18, 42), Array(Array(1), Array(45))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) AS A (s)" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,[12, 45]", "2,[18]", "2,[87]", "3,[1]", "3,[45]") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[Int], Array[Array[Int]])], + sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) AS A (s)", + expectedResults = List("1,[12, 45]", "2,[18]", "2,[87]", "3,[1]", "3,[45]"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c + ) } @TestTemplate @@ -97,42 +125,16 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array((13, "41.6"), (14, "45.2136"))), (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, b, s, t FROM T, UNNEST(T.b) AS A (s, t) WHERE s > 13" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) - } - - @TestTemplate - def testUnnestMultiSetFromCollectResult(): Unit = { - val data = List( - (1, 1, (12, "45.6")), - (2, 2, (12, "45.612")), - (3, 2, (13, "41.6")), - (4, 3, (14, "45.2136")), - (5, 3, (18, "42.6"))) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |WITH T1 AS (SELECT b, COLLECT(c) as `set` FROM T GROUP BY b) - |SELECT b, id, point FROM T1, UNNEST(T1.`set`) AS A(id, point) WHERE b < 3 - """.stripMargin - val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] - val sink = new TestingRetractSink - result.addSink(sink) - env.execute() - - val expected = List("1,12,45.6", "2,12,45.612", "2,13,41.6") - assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = "SELECT a, b, s, t FROM T, UNNEST(T.b) AS A (s, t) WHERE s > 13", + expectedResults = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -150,27 +152,26 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (7, "8", "Hello World") ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |WITH T1 AS (SELECT a, COLLECT(b) as `set` FROM T GROUP BY a) - |SELECT a, s FROM T1 LEFT JOIN UNNEST(T1.`set`) AS A(s) ON TRUE WHERE a < 5 - """.stripMargin - val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] - val sink = new TestingRetractSink - result.addSink(sink).setParallelism(1) - env.execute() - - val expected = List( - "1,1", - "1,2", - "2,2", - "3,null", - "4,4" + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, String, String)], + sqlQuery = """ + |WITH T1 AS (SELECT a, COLLECT(b) as `set` FROM T GROUP BY a) + |SELECT a, s FROM T1 LEFT JOIN UNNEST(T1.`set`) AS A(s) ON TRUE WHERE a < 5 + """.stripMargin, + expectedResults = List( + "1,1", + "1,2", + "2,2", + "3,null", + "4,4" + ), + isRetract = true, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c ) - assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) } @TestTemplate @@ -210,17 +211,17 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (3, 2L, Array("Hello world", "x")) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) as A (s)" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,Hi", "1,w", "2,Hello", "2,k", "3,Hello world", "3,x") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Long, Array[String])], + sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) as A (s)", + expectedResults = List("1,Hi", "1,w", "2,Hello", "2,k", "3,Hello world", "3,x"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c + ) } @TestTemplate @@ -255,27 +256,18 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Array("a", "b", "c"), Array[TypeInformation[_]](Types.INT, Types.LONG, Types.MAP(Types.STRING, Types.STRING)) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - val sqlQuery = "SELECT a, b, v FROM T CROSS JOIN UNNEST(c) as f (k, v)" - val result = tEnv.sqlQuery(sqlQuery) - - TestSinkUtil.addValuesSink( - tEnv, - "MySink", - List("a", "b", "v"), - List(DataTypes.INT, DataTypes.BIGINT, DataTypes.STRING), - ChangelogMode.all() - ) - result.executeInsert("MySink").await() - - val expected = - List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31") - assertThat( - TestValuesTableFactory - .getResultsAsStrings("MySink") - .sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = typeInfo, + sqlQuery = "SELECT a, b, v FROM T CROSS JOIN UNNEST(c) as f (k, v)", + expectedResults = List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c + ) } @TestTemplate @@ -286,24 +278,25 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, b, x, y " + - "FROM " + - " (SELECT a, b FROM T WHERE a < 3) as tf, " + - " UNNEST(tf.b) as A (x, y) " + - "WHERE x > a" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List( - "1,[12,45.6, 2,45.612],12,45.6", - "1,[12,45.6, 2,45.612],2,45.612", - "2,[13,41.6, 1,45.2136],13,41.6") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = """ + |SELECT a, b, x, y + |FROM + | (SELECT a, b FROM T WHERE a < 3) as tf, + | UNNEST(tf.b) as A (x, y) + |WHERE x > a + |""".stripMargin, + expectedResults = List( + "1,[12,45.6, 2,45.612],12,45.6", + "1,[12,45.6, 2,45.612],2,45.612", + "2,[13,41.6, 1,45.2136],13,41.6"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -313,17 +306,16 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array((13, "41.6"), (14, "45.2136"))), (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, b, A._1, A._2 FROM T, UNNEST(T.b) AS A where A._1 > 13" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = "SELECT a, b, A._1, A._2 FROM T, UNNEST(T.b) AS A where A._1 > 13", + expectedResults = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -333,64 +325,64 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array((13, "41.6"), (14, "45.2136"))), (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("MyTable", t) - - val sqlQuery = - """ - |SELECT * FROM ( - | SELECT a, b1, b2 FROM - | (SELECT a, b FROM MyTable) T - | CROSS JOIN - | UNNEST(T.b) as S(b1, b2) - | WHERE S.b1 >= 12 - | ) tmp - |WHERE b2 <> '42.6' - """.stripMargin - - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,12,45.612", "1,12,45.6", "2,13,41.6", "2,14,45.2136") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = """ + |SELECT * FROM ( + | SELECT a, b1, b2 FROM + | (SELECT a, b FROM T) T2 + | CROSS JOIN + | UNNEST(T2.b) as S(b1, b2) + | WHERE S.b1 >= 12 + | ) tmp + |WHERE b2 <> '42.6' + |""".stripMargin, + expectedResults = List("1,12,45.612", "1,12,45.6", "2,13,41.6", "2,14,45.2136"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate def testUnnestWithValuesStream(): Unit = { - val sqlQuery = "SELECT * FROM UNNEST(ARRAY[1,2,3])" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1", "2", "3") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = List(1), + typeInfo = createTypeInformation[Int], + sqlQuery = "SELECT * FROM UNNEST(ARRAY[1,2,3])", + expectedResults = List("1", "2", "3"), + isRetract = false, + parallelism = -1, + fieldNames = 'dummy + ) } @TestTemplate def testUnnestWithValuesStream2(): Unit = { - val sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3])" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("a,1", "a,2", "a,3") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = List(1), + typeInfo = createTypeInformation[Int], + sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3])", + expectedResults = List("a,1", "a,2", "a,3"), + isRetract = false, + parallelism = -1, + fieldNames = 'dummy + ) } @TestTemplate def testUnnestWithOrdinalityWithValuesStream(): Unit = { - val sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3]) WITH ORDINALITY" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("a,1,1", "a,2,2", "a,3,3") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = List(1), + typeInfo = createTypeInformation[Int], + sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3]) WITH ORDINALITY", + expectedResults = List("a,1,1", "a,2,2", "a,3,3"), + isRetract = false, + parallelism = -1, + fieldNames = 'dummy + ) } @TestTemplate @@ -400,20 +392,19 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array(41, 5)), (3, Array(18, 42)) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = """ - |SELECT a, number, ordinality - |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,12,1", "1,45,2", "2,41,1", "2,5,2", "3,18,1", "3,42,2") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[Int])], + sqlQuery = """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin, + expectedResults = List("1,12,1", "1,45,2", "2,41,1", "2,5,2", "3,18,1", "3,42,2"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -424,17 +415,17 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (3, 2L, Array("Hello world", "x")) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY as A (s, o)" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,Hi,1", "1,w,2", "2,Hello,1", "2,k,2", "3,Hello world,1", "3,x,2") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Long, Array[String])], + sqlQuery = "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY as A (s, o)", + expectedResults = List("1,Hi,1", "1,w,2", "2,Hello,1", "2,k,2", "3,Hello world,1", "3,x,2"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c + ) } @TestTemplate @@ -444,31 +435,29 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array(Array(4, 5), Array(6, 7))), (3, Array(Array(8))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'nested_array) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |SELECT id, array_val, array_pos, `element`, element_pos - |FROM T - |CROSS JOIN UNNEST(nested_array) WITH ORDINALITY AS A(array_val, array_pos) - |CROSS JOIN UNNEST(array_val) WITH ORDINALITY AS B(`element`, element_pos) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List( - "1,[1, 2],1,1,1", - "1,[1, 2],1,2,2", - "1,[3],2,3,1", - "2,[4, 5],1,4,1", - "2,[4, 5],1,5,2", - "2,[6, 7],2,6,1", - "2,[6, 7],2,7,2", - "3,[8],1,8,1") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[Array[Int]])], + sqlQuery = """ + |SELECT id, array_val, array_pos, `element`, element_pos + |FROM T + |CROSS JOIN UNNEST(nested_array) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(array_val) WITH ORDINALITY AS B(`element`, element_pos) + |""".stripMargin, + expectedResults = List( + "1,[1, 2],1,1,1", + "1,[1, 2],1,2,2", + "1,[3],2,3,1", + "2,[4, 5],1,4,1", + "2,[4, 5],1,5,2", + "2,[6, 7],2,6,1", + "2,[6, 7],2,7,2", + "3,[8],1,8,1"), + isRetract = false, + parallelism = -1, + fieldNames = 'id, + 'nested_array + ) } @TestTemplate @@ -479,27 +468,26 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, 2, "World"), (3, 3, "Hello world") ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b, 'c) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |WITH T1 AS (SELECT a, COLLECT(c) as words FROM T GROUP BY a) - |SELECT a, word, pos - |FROM T1 CROSS JOIN UNNEST(words) WITH ORDINALITY AS A(word, pos) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] - val sink = new TestingRetractSink - result.addSink(sink) - env.execute() - - val expected = List( - "1,Hi,1", - "1,Hello,2", - "2,World,1", - "3,Hello world,1" + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Int, String)], + sqlQuery = """ + |WITH T1 AS (SELECT a, COLLECT(c) as words FROM T GROUP BY a) + |SELECT a, word, pos + |FROM T1 CROSS JOIN UNNEST(words) WITH ORDINALITY AS A(word, pos) + |""".stripMargin, + expectedResults = List( + "1,Hi,1", + "1,Hello,2", + "2,World,1", + "3,Hello world,1" + ), + isRetract = true, + parallelism = -1, + fieldNames = 'a, + 'b, + 'c ) - assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) } @TestTemplate @@ -525,26 +513,24 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Array("id", "map_data"), Array[TypeInformation[_]](Types.INT, Types.MAP(Types.STRING, Types.STRING)) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'map_data) - tEnv.createTemporaryView("T", t) - val sqlQuery = """ - |SELECT id, k, v, pos - |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val resultsWithoutordinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) - val expected = List("1,a,10", "1,b,11", "2,c,20", "2,d,21") - - assertThat(resultsWithoutordinality.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = typeInfo, + sqlQuery = """ + |SELECT id, k, v, pos + |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) + |""".stripMargin, + expectedResults = List("1,a,10,1", "1,b,11,2", "2,c,20,2", "2,d,21,1"), + isRetract = false, + parallelism = 1, + fieldNames = 'id, + 'map_data + ) } - def testUnnestForMapOfRowsWitOrdinality(): Unit = { + @TestTemplate + def testUnnestForMapOfRowsWithOrdinality(): Unit = { val data = List( Row.of( Int.box(1), { @@ -574,30 +560,23 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Types.INT, Types.MAP(Types.ROW(Types.STRING, Types.STRING), Types.ROW(Types.INT()))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - val sqlQuery = "SELECT a, k, v, o FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY as f (k, v, o)" - val result = tEnv.sqlQuery(sqlQuery) - TestSinkUtil.addValuesSink( - tEnv, - "MySink", - List("a", "k", "v", "o"), - List( - DataTypes.INT, - DataTypes.ROW(DataTypes.STRING(), DataTypes.STRING()), - DataTypes.ROW(DataTypes.INT()), - DataTypes.INT.notNull()), - ChangelogMode.all() - ) - result.executeInsert("MySink").await() - - val expected = - List("1,a,a,10", "1,b,b,11", "2,c,c,20", "3,d,d,30", "3,e,e,31") - val resultWithoutOrd = assertAndRemoveOrdinality( - TestValuesTableFactory.getResultsAsStrings("MySink").sorted.toList, - 2) - assertThat(resultWithoutOrd).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = typeInfo, + sqlQuery = "SELECT a, k, v, o FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY as f (k, v, o)", + expectedResults = List( + "1,a,a,10,1", + "1,b,b,11,2", + "2,c,c,20,1", + "3,d,d,30,1", + "3,e,e,31,2" + ), + isRetract = false, + parallelism = 1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -628,35 +607,32 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Types.OBJECT_ARRAY(Types.STRING), Types.MAP(Types.STRING, Types.STRING)) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'array_data, 'map_data) - tEnv.createTemporaryView("T", t) - val sqlQuery = - """ - |SELECT id, array_val, array_pos, map_key, map_val, map_pos - |FROM T - |CROSS JOIN UNNEST(array_data) WITH ORDINALITY AS A(array_val, array_pos) - |CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS B(map_key, map_val, map_pos) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val resultsWithoutOrdinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) - val expected = List( - "1,a,1,x,10", - "1,a,1,y,20", - "1,b,2,x,10", - "1,b,2,y,20", - "2,c,1,z,30", - "2,c,1,w,40", - "2,d,2,z,30", - "2,d,2,w,40" + assertUnnest( + testData = data, + typeInfo = typeInfo, + sqlQuery = """ + |SELECT id, array_val, array_pos, map_key, map_val, map_pos + |FROM T + |CROSS JOIN UNNEST(array_data) WITH ORDINALITY AS A(array_val, array_pos) + |CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS B(map_key, map_val, map_pos) + |""".stripMargin, + expectedResults = List( + "1,a,1,x,10,1", + "1,a,1,y,20,2", + "1,b,2,x,10,1", + "1,b,2,y,20,2", + "2,c,1,z,30,1", + "2,c,1,w,40,2", + "2,d,2,z,30,1", + "2,d,2,w,40,2" + ), + isRetract = false, + parallelism = -1, + fieldNames = 'id, + 'array_data, + 'map_data ) - - assertThat(resultsWithoutOrdinality.sorted).isEqualTo(expected.sorted) } @TestTemplate @@ -664,20 +640,19 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo val data = List( (1, Array[Int]()) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = """ - |SELECT a, number, ordinality - |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List() - assertThat(sink.getAppendResults.sorted).isEqualTo(expected) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[Int])], + sqlQuery = """ + |SELECT a, number, ordinality + |FROM T CROSS JOIN UNNEST(b) WITH ORDINALITY AS t(number, ordinality) + |""".stripMargin, + expectedResults = List(), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -703,23 +678,20 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Array("id", "map_data"), Array[TypeInformation[_]](Types.INT, Types.MAP(Types.STRING, Types.STRING)) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'id, 'map_data) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |SELECT id, k, v, pos - |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) - |""".stripMargin - val result = tEnv.sqlQuery(sqlQuery).toDataStream - - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - val resultsWithoutordinality = assertAndRemoveOrdinality(sink.getAppendResults, 2) - val expected = List("1,a,10", "1,b,null", "2,c,20", "2,d,null") - assertThat(resultsWithoutordinality.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = typeInfo, + sqlQuery = """ + |SELECT id, k, v, pos + |FROM T CROSS JOIN UNNEST(map_data) WITH ORDINALITY AS f(k, v, pos) + |""".stripMargin, + expectedResults = List("1,a,10,1", "1,b,null,2", "2,c,20,2", "2,d,null,1"), + isRetract = false, + parallelism = 1, + fieldNames = 'id, + 'map_data + ) } @TestTemplate @@ -728,21 +700,19 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array((20, "41.6"), (14, "45.2136"))), (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - "SELECT a, b, s, t, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A(s, t, o)" - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List( - "2,[20,41.6, 14,45.2136],20,41.6,1", - "2,[20,41.6, 14,45.2136],14,45.2136,2", - "3,[18,42.6],18,42.6,1") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = "SELECT a, b, s, t, o FROM T, UNNEST(T.b) WITH ORDINALITY AS A(s, t, o)", + expectedResults = List( + "2,[20,41.6, 14,45.2136],20,41.6,1", + "2,[20,41.6, 14,45.2136],14,45.2136,2", + "3,[18,42.6],18,42.6,1"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -752,28 +722,25 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, Array((13, "41.6"), (14, "45.2136"))), (3, Array((18, "42.6"))) ) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("MyTable", t) - - val sqlQuery = - """ - |SELECT * FROM ( - | SELECT a, b1, b2, ord FROM - | (SELECT a, b FROM MyTable) T - | CROSS JOIN - | UNNEST(T.b) WITH ORDINALITY as S(b1, b2, ord) - | WHERE S.b1 >= 12 - | ) tmp - |WHERE b2 <> '42.6' AND ord <> 2 - """.stripMargin - - val result = tEnv.sqlQuery(sqlQuery).toDataStream - val sink = new TestingAppendSink - result.addSink(sink) - env.execute() - - val expected = List("1,12,45.6,1", "2,13,41.6,1") - assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, Array[(Int, String)])], + sqlQuery = """ + |SELECT * FROM ( + | SELECT a, b1, b2, ord FROM + | (SELECT a, b FROM T) T2 + | CROSS JOIN + | UNNEST(T2.b) WITH ORDINALITY as S(b1, b2, ord) + | WHERE S.b1 >= 12 + | ) tmp + |WHERE b2 <> '42.6' AND ord <> 2 + |""".stripMargin, + expectedResults = List("1,12,45.6,1", "2,13,41.6,1"), + isRetract = false, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -784,22 +751,20 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo (2, (13, "41.6")), (3, (14, "45.2136")), (3, (18, "42.6"))) - val t = StreamingEnvUtil.fromCollection(env, data).toTable(tEnv, 'a, 'b) - tEnv.createTemporaryView("T", t) - - val sqlQuery = - """ - |WITH T1 AS (SELECT a, COLLECT(b) as `set` FROM T GROUP BY a) - |SELECT a, id, point, o FROM T1 - |CROSS JOIN UNNEST(T1.`set`) WITH ORDINALITY AS A(id, point, o) WHERE a < 3 - """.stripMargin - val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] - val sink = new TestingRetractSink - result.addSink(sink) - env.execute() - - val expected = List("1,12,45.6,1", "2,12,45.612,1", "2,13,41.6,2") - assertThat(sink.getRetractResults.sorted).isEqualTo(expected.sorted) + assertUnnest( + testData = data, + typeInfo = createTypeInformation[(Int, (Int, String))], + sqlQuery = """ + |WITH T1 AS (SELECT a, COLLECT(b) as `set` FROM T GROUP BY a) + |SELECT a, id, point, o FROM T1 + |CROSS JOIN UNNEST(T1.`set`) WITH ORDINALITY AS A(id, point, o) WHERE a < 3 + |""".stripMargin, + expectedResults = List("1,12,45.6,1", "2,12,45.612,1", "2,13,41.6,2"), + isRetract = true, + parallelism = -1, + fieldNames = 'a, + 'b + ) } @TestTemplate @@ -846,20 +811,4 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo val expectedFieldNames = List("x", "y", "ord") assertThat(fieldNames).isEqualTo(expectedFieldNames) } - - /* Utility for maps to assert that ordinality is within range and remove it from output. - * Necessary since maps are not ordered */ - def assertAndRemoveOrdinality(results: List[String], maxOrdinality: Int): List[String] = { - results.foreach { - result => - val columns = result.split(",") - val ordinality = columns.last.toInt - assert( - ordinality >= 1 && ordinality <= maxOrdinality, - s"Ordinality $ordinality out of range") - } - - results.map(_.split(",").dropRight(1).mkString(",")) - } - } From 8d964daf07e8f71637f5d9fb9371342e582a6bd5 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Sat, 15 Feb 2025 15:18:13 +0100 Subject: [PATCH 11/15] [FLINK-37267][table] Use .asJava instead of HashMap --- .../runtime/batch/sql/UnnestITCase.scala | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala index f4fdbe6928dee..669c9962487a9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala @@ -448,25 +448,22 @@ class UnnestITCase extends BatchTestBase { def testUnnestForMapOfRowsWithOrdinality(): Unit = { val data = List( row( - 1, { - val map = new java.util.HashMap[Row, Row]() - map.put(Row.of("a", "a"), Row.of(10: Integer)) - map.put(Row.of("b", "b"), Row.of(11: Integer)) - map - }), + 1, + Map( + Row.of("a", "a") -> Row.of(10: Integer), + Row.of("b", "b") -> Row.of(11: Integer) + ).asJava), row( - 2, { - val map = new java.util.HashMap[Row, Row]() - map.put(Row.of("c", "c"), Row.of(20: Integer)) - map - }), + 2, + Map( + Row.of("c", "c") -> Row.of(20: Integer) + ).asJava), row( - 3, { - val map = new java.util.HashMap[Row, Row]() - map.put(Row.of("d", "d"), Row.of(30: Integer)) - map.put(Row.of("e", "e"), Row.of(31: Integer)) - map - }) + 3, + Map( + Row.of("d", "d") -> Row.of(30: Integer), + Row.of("e", "e") -> Row.of(31: Integer) + ).asJava) ) registerCollection( From 143e6adb0016cedebc9c190af1e6479ac2d8b9a2 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Mon, 17 Feb 2025 12:38:25 +0100 Subject: [PATCH 12/15] [FLINK-37267][table] Update unnest plans with column names --- .../apache/flink/table/planner/plan/batch/sql/UnnestTest.xml | 4 ++-- .../planner/plan/rules/logical/LogicalUnnestRuleTest.xml | 4 ++-- .../apache/flink/table/planner/plan/stream/sql/UnnestTest.xml | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index 63f245d7b3770..be94f318e9b54 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -450,7 +450,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) ($0, 13)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ordinality], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)], joinType=[INNER], condition=[>($0, 13)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -472,7 +472,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) ($0, 1)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ordinality], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)], joinType=[INNER], condition=[>($0, 1)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index fa097c67b9648..eef1cc5eeef38 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -483,7 +483,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1], o=[$2]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, VARCHAR(2147483647) f1, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)]) ]]> @@ -508,7 +508,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, VARCHAR(2147483647) f1, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index bb3de22851a70..6f5fa2678b3f6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -462,7 +462,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) ($0, 13)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ordinality], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)], joinType=[INNER], condition=[>($0, 13)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -484,7 +484,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) ($0, 1)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ordinality], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)], joinType=[INNER], condition=[>($0, 1)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> From 5c72d20acdb392f8f82d1b0b9d813206064d6774 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 19 Feb 2025 10:11:06 +0100 Subject: [PATCH 13/15] [FLINK-37267][table] Use JoinedRowData instead of field getters --- .../UnnestRowsWithOrdinalityFunction.java | 32 ++----------------- 1 file changed, 3 insertions(+), 29 deletions(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java index fa6276347bc29..6bd738a09e7c7 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java @@ -23,6 +23,7 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.MapData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.utils.JoinedRowData; import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -66,7 +67,6 @@ public static final class CollectionUnnestWithOrdinalityFunction private static final long serialVersionUID = 1L; private final ArrayData.ElementGetter elementGetter; - private RowData.FieldGetter[] fieldGetters = null; public CollectionUnnestWithOrdinalityFunction( SpecializedContext context, @@ -74,15 +74,6 @@ public CollectionUnnestWithOrdinalityFunction( ArrayData.ElementGetter elementGetter) { super(context, elementType, true); this.elementGetter = elementGetter; - - if (elementType instanceof RowType) { - /* When unnesting a collection, according to Calcite's implementation, - row(a,b) unnests to a row(a, b, ordinality) and not to (row(a,b), ordinality). - That means, if we are unnesting a row, we need field getters - to be able to extract all field values */ - RowType rowType = (RowType) elementType; - this.fieldGetters = createFieldGetters(rowType); - } } public void eval(ArrayData arrayData) { @@ -95,29 +86,12 @@ public void eval(MapData mapData) { private void collectWithOrdinality(Object element, int position) { if (element instanceof RowData) { - RowData innerRow = (RowData) element; - int arity = innerRow.getArity(); - GenericRowData outRow = new GenericRowData(arity + 1); - - for (int i = 0; i < arity; i++) { - outRow.setField(i, fieldGetters[i].getFieldOrNull(innerRow)); - } - - outRow.setField(arity, position); - collect(outRow); + RowData row = (RowData) element; + collect(new JoinedRowData(row.getRowKind(), row, GenericRowData.of(position))); } else { collect(GenericRowData.of(element, position)); } } - - private RowData.FieldGetter[] createFieldGetters(RowType rowType) { - int fieldCount = rowType.getFieldCount(); - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[fieldCount]; - for (int i = 0; i < fieldCount; i++) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - return fieldGetters; - } } /** Table function that unwraps the elements of a map with ordinality. */ From 767ac268fb52bb86792e36c1b5992312901962dd Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 19 Feb 2025 10:26:45 +0100 Subject: [PATCH 14/15] [FLINK-37267][table] Remove parallelism from assertUnnest --- .../runtime/stream/sql/UnnestITCase.scala | 34 ++----------------- 1 file changed, 2 insertions(+), 32 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index ba8eb7480377e..0a4eb3bfcc93c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -23,13 +23,12 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.expressions.Expression import org.apache.flink.table.legacy.api.Types -import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset +import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.table.utils.LegacyRowExtension import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension import org.apache.flink.types.Row - import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.TestTemplate import org.junit.jupiter.api.extension.{ExtendWith, RegisterExtension} @@ -48,7 +47,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery: String, expectedResults: List[String], isRetract: Boolean, - parallelism: Int, fieldNames: Expression*): Unit = { val t = StreamingEnvUtil.fromCollection(env, testData)(typeInfo).toTable(tEnv, fieldNames: _*) tEnv.createTemporaryView("T", t) @@ -56,11 +54,7 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo val result = tEnv.sqlQuery(sqlQuery) if (isRetract) { val sink = new TestingRetractSink - if (parallelism > 0) { - result.toRetractStream[Row].addSink(sink).setParallelism(parallelism) - } else { - result.toRetractStream[Row].addSink(sink) - } + result.toRetractStream[Row].addSink(sink) env.execute() assertThat(sink.getRetractResults.sorted).isEqualTo(expectedResults.sorted) } else { @@ -91,7 +85,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "3,[18, 42],18", "3,[18, 42],42"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -111,7 +104,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) AS A (s)", expectedResults = List("1,[12, 45]", "2,[18]", "2,[87]", "3,[1]", "3,[45]"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -131,7 +123,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, b, s, t FROM T, UNNEST(T.b) AS A (s, t) WHERE s > 13", expectedResults = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -167,7 +158,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "4,4" ), isRetract = true, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -217,7 +207,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) as A (s)", expectedResults = List("1,Hi", "1,w", "2,Hello", "2,k", "3,Hello world", "3,x"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -263,7 +252,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, b, v FROM T CROSS JOIN UNNEST(c) as f (k, v)", expectedResults = List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -293,7 +281,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "1,[12,45.6, 2,45.612],2,45.612", "2,[13,41.6, 1,45.2136],13,41.6"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -312,7 +299,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, b, A._1, A._2 FROM T, UNNEST(T.b) AS A where A._1 > 13", expectedResults = List("2,[13,41.6, 14,45.2136],14,45.2136", "3,[18,42.6],18,42.6"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -340,7 +326,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,12,45.612", "1,12,45.6", "2,13,41.6", "2,14,45.2136"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -354,7 +339,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT * FROM UNNEST(ARRAY[1,2,3])", expectedResults = List("1", "2", "3"), isRetract = false, - parallelism = -1, fieldNames = 'dummy ) } @@ -367,7 +351,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3])", expectedResults = List("a,1", "a,2", "a,3"), isRetract = false, - parallelism = -1, fieldNames = 'dummy ) } @@ -380,7 +363,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3]) WITH ORDINALITY", expectedResults = List("a,1,1", "a,2,2", "a,3,3"), isRetract = false, - parallelism = -1, fieldNames = 'dummy ) } @@ -401,7 +383,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,12,1", "1,45,2", "2,41,1", "2,5,2", "3,18,1", "3,42,2"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -421,7 +402,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo sqlQuery = "SELECT a, s, o FROM T, UNNEST(T.c) WITH ORDINALITY as A (s, o)", expectedResults = List("1,Hi,1", "1,w,2", "2,Hello,1", "2,k,2", "3,Hello world,1", "3,x,2"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -454,7 +434,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "2,[6, 7],2,7,2", "3,[8],1,8,1"), isRetract = false, - parallelism = -1, fieldNames = 'id, 'nested_array ) @@ -483,7 +462,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "3,Hello world,1" ), isRetract = true, - parallelism = -1, fieldNames = 'a, 'b, 'c @@ -523,7 +501,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,a,10,1", "1,b,11,2", "2,c,20,2", "2,d,21,1"), isRetract = false, - parallelism = 1, fieldNames = 'id, 'map_data ) @@ -573,7 +550,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "3,e,e,31,2" ), isRetract = false, - parallelism = 1, fieldNames = 'a, 'b ) @@ -628,7 +604,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "2,d,2,w,40,2" ), isRetract = false, - parallelism = -1, fieldNames = 'id, 'array_data, 'map_data @@ -649,7 +624,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List(), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -688,7 +662,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,a,10,1", "1,b,null,2", "2,c,20,2", "2,d,null,1"), isRetract = false, - parallelism = 1, fieldNames = 'id, 'map_data ) @@ -709,7 +682,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo "2,[20,41.6, 14,45.2136],14,45.2136,2", "3,[18,42.6],18,42.6,1"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -737,7 +709,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,12,45.6,1", "2,13,41.6,1"), isRetract = false, - parallelism = -1, fieldNames = 'a, 'b ) @@ -761,7 +732,6 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo |""".stripMargin, expectedResults = List("1,12,45.6,1", "2,12,45.612,1", "2,13,41.6,2"), isRetract = true, - parallelism = -1, fieldNames = 'a, 'b ) From 77277e04bd16b6564702c5aa00e060751559dcc2 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 19 Feb 2025 14:16:30 +0100 Subject: [PATCH 15/15] [FLINK-37267][table] Column names to EXPR$0 and ORDINALITY --- .../planner/plan/batch/sql/UnnestTest.xml | 28 +++++++++---------- .../rules/logical/LogicalUnnestRuleTest.xml | 18 ++++++------ .../planner/plan/stream/sql/UnnestTest.xml | 28 +++++++++---------- .../runtime/stream/sql/UnnestITCase.scala | 3 +- .../table/UnnestRowsFunctionBase.java | 4 +-- 5 files changed, 41 insertions(+), 40 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index be94f318e9b54..4645f60d6c84a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -370,7 +370,7 @@ LogicalProject(val=[$0], pos=[$1]) @@ -392,8 +392,8 @@ LogicalProject(a=[$0], number=[$2], ordinality=[$3]) @@ -425,9 +425,9 @@ LogicalProject(id=[$0], array_val=[$2], array_pos=[$3], elem=[$4], element_pos=[ @@ -450,7 +450,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) ($0, 13)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ORDINALITY], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)], joinType=[INNER], condition=[>($0, 13)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -472,7 +472,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) ($0, 1)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ORDINALITY], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)], joinType=[INNER], condition=[>($0, 1)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -494,8 +494,8 @@ LogicalProject(id=[$0], k=[$2], v=[$3], pos=[$4]) @@ -523,8 +523,8 @@ LogicalProject(a=[$0], word=[$2], pos=[$3]) ($0, 10), <($1, 3))]) +Calc(select=[a, EXPR$0 AS number, ORDINALITY AS ordinality]) ++- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,EXPR$0,ORDINALITY], rowType=[RecordType(INTEGER a, INTEGER ARRAY b, INTEGER EXPR$0, INTEGER ORDINALITY)], joinType=[INNER], condition=[AND(>($0, 10), <($1, 3))]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index eef1cc5eeef38..5bafc282773e0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -418,7 +418,7 @@ LogicalProject(a=[$0], number=[$2], ordinality=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(number=[$0], ordinality=[$1]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER EXPR$0, INTEGER ORDINALITY)]) ]]> @@ -454,9 +454,9 @@ LogicalProject(id=[$0], array_val=[$2], array_pos=[$3], elem=[$4], element_pos=[ :- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(array_val=[$0], array_pos=[$1]) - : +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array)], rowType=[RecordType:peek_no_expand(INTEGER ARRAY f0, INTEGER ordinality)]) + : +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.nested_array)], rowType=[RecordType:peek_no_expand(INTEGER ARRAY EXPR$0, INTEGER ORDINALITY)]) +- LogicalProject(elem=[$0], element_pos=[$1]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor1.array_val)], rowType=[RecordType:peek_no_expand(INTEGER EXPR$0, INTEGER ORDINALITY)]) ]]> @@ -483,7 +483,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1], o=[$2]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)]) ]]> @@ -508,7 +508,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)]) ]]> @@ -533,7 +533,7 @@ LogicalProject(id=[$0], k=[$2], v=[$3], pos=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(k=[$0], v=[$1], pos=[$2]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.map_data)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, VARCHAR(2147483647) f1, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.map_data)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, VARCHAR(2147483647) f1, INTEGER ORDINALITY)]) ]]> @@ -566,7 +566,7 @@ LogicalProject(a=[$0], word=[$2], pos=[$3]) : +- LogicalProject(a=[$0], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(word=[$0], pos=[$1]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.words)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.words)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) EXPR$0, INTEGER ORDINALITY)]) ]]> @@ -598,7 +598,7 @@ LogicalProject(a=[$0], number=[$2], ordinality=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(number=[$0], ordinality=[$1]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER EXPR$0, INTEGER ORDINALITY)]) ]]> @@ -634,7 +634,7 @@ LogicalProject(a=[$0], number=[$1], ordinality=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(number=[$0], ordinality=[$1]) - +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER f0, INTEGER ordinality)]) + +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER EXPR$0, INTEGER ORDINALITY)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index 6f5fa2678b3f6..77d1e1a758bbe 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -382,7 +382,7 @@ LogicalProject(val=[$0], pos=[$1]) @@ -404,8 +404,8 @@ LogicalProject(a=[$0], number=[$2], ordinality=[$3]) @@ -437,9 +437,9 @@ LogicalProject(id=[$0], array_val=[$2], array_pos=[$3], elem=[$4], element_pos=[ @@ -462,7 +462,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3], o=[$4]) ($0, 13)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ORDINALITY], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)], joinType=[INNER], condition=[>($0, 13)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -484,7 +484,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3], ORDINALITY=[$4]) ($0, 1)]) +Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,_1,_2,ORDINALITY], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2, INTEGER ORDINALITY)], joinType=[INNER], condition=[>($0, 1)]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -506,8 +506,8 @@ LogicalProject(id=[$0], k=[$2], v=[$3], pos=[$4]) @@ -535,8 +535,8 @@ LogicalProject(a=[$0], word=[$2], pos=[$3]) ($0, 10), <($1, 3))]) +Calc(select=[a, EXPR$0 AS number, ORDINALITY AS ordinality]) ++- Correlate(invocation=[$UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b)], correlate=[table($UNNEST_ROWS_WITH_ORDINALITY$1($cor0.b))], select=[a,b,EXPR$0,ORDINALITY], rowType=[RecordType(INTEGER a, INTEGER ARRAY b, INTEGER EXPR$0, INTEGER ORDINALITY)], joinType=[INNER], condition=[AND(>($0, 10), <($1, 3))]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 0a4eb3bfcc93c..ec8e29e24c0b4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -23,12 +23,13 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.expressions.Expression import org.apache.flink.table.legacy.api.Types +import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset -import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.table.utils.LegacyRowExtension import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension import org.apache.flink.types.Row + import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.TestTemplate import org.junit.jupiter.api.extension.{ExtendWith, RegisterExtension} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java index a4b73fad12674..ea340adf2e792 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java @@ -119,7 +119,7 @@ public static LogicalType wrapWithOrdinality(LogicalType baseType) { rowType.getFields().stream(), Stream.of( new RowType.RowField( - "ordinality", + "ORDINALITY", DataTypes.INT().notNull().getLogicalType()))) .collect(Collectors.toList())); } else { @@ -127,7 +127,7 @@ public static LogicalType wrapWithOrdinality(LogicalType baseType) { return RowType.of( false, new LogicalType[] {baseType, DataTypes.INT().notNull().getLogicalType()}, - new String[] {"f0", "ordinality"}); + new String[] {"EXPR$0", "ORDINALITY"}); } }