twalthr commented on code in PR #26113:
URL: https://github.com/apache/flink/pull/26113#discussion_r1959540183


##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala:
##########
@@ -312,71 +306,509 @@ 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)
+    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
+    )
+  }
 
-    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()
+  @TestTemplate
+  def testUnnestArrayOfRowsWithNestedFilter(): 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")))
+    )
+    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
+    )
+  }
 
-    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 testUnnestWithValuesStream(): Unit = {
+    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 = {
+    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 = {
+    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
+  def testUnnestArrayWithOrdinality(): Unit = {
+    val data = List(
+      (1, Array(12, 45)),
+      (2, Array(41, 5)),
+      (3, Array(18, 42))
+    )
+    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
-  def testUnnestWithNestedFilter(): Unit = {
+  def testUnnestFromTableWithOrdinality(): Unit = {
+    val data = List(
+      (1, 1L, Array("Hi", "w")),
+      (2, 2L, Array("Hello", "k")),
+      (3, 2L, Array("Hello world", "x"))
+    )
+
+    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
+  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)))
+    )
+    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
+  def testUnnestMultisetWithOrdinality(): Unit = {
+    val data = List(
+      (1, 1, "Hi"),
+      (1, 2, "Hello"),
+      (2, 2, "World"),
+      (3, 3, "Hello world")
+    )
+    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
+    )
+  }
+
+  @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))
+    )
+
+    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,

Review Comment:
   Is this parallelism important for the behavior of the test? if not, we can 
run all tests with the same value and remove this parameter from 
`assertUnnest()`



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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;
+
+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 {
+
+    public UnnestRowsFunctionBase(boolean withOrdinality) {
+        super(
+                withOrdinality
+                        ? 
BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY
+                        : 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 elementType;
+        switch (logicalType.getTypeRoot()) {
+            case ARRAY:
+                elementType = ((ArrayType) logicalType).getElementType();
+                break;
+            case MULTISET:
+                elementType = ((MultisetType) logicalType).getElementType();
+                break;
+            case MAP:
+                MapType mapType = (MapType) logicalType;
+                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;
+            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(
+                    false,
+                    new LogicalType[] {baseType, 
DataTypes.INT().notNull().getLogicalType()},
+                    new String[] {"f0", "ordinality"});

Review Comment:
   For PTF [we changed the 
default](https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/SystemTypeInference.java#L272)
 from `f0` to `EXPR$0`. We should do the same here.
   



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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;
+
+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 {
+
+    public UnnestRowsFunctionBase(boolean withOrdinality) {
+        super(
+                withOrdinality
+                        ? 
BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY
+                        : 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 elementType;
+        switch (logicalType.getTypeRoot()) {
+            case ARRAY:
+                elementType = ((ArrayType) logicalType).getElementType();
+                break;
+            case MULTISET:
+                elementType = ((MultisetType) logicalType).getElementType();
+                break;
+            case MAP:
+                MapType mapType = (MapType) logicalType;
+                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;
+            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(
+                    false,
+                    new LogicalType[] {baseType, 
DataTypes.INT().notNull().getLogicalType()},
+                    new String[] {"f0", "ordinality"});

Review Comment:
   ```suggestion
                       new String[] {"EXPR$0", "ORDINALITY"});
   ```



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsWithOrdinalityFunction.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.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.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(true);
+    }
+
+    @Override
+    protected UserDefinedFunction createCollectionUnnestFunction(
+            SpecializedContext context,
+            LogicalType elementType,
+            ArrayData.ElementGetter elementGetter) {
+        return new CollectionUnnestWithOrdinalityFunction(
+                context, wrapWithOrdinality(elementType), elementGetter);
+    }
+
+    @Override
+    protected UserDefinedFunction createMapUnnestFunction(
+            SpecializedContext context,
+            RowType keyValTypes,
+            ArrayData.ElementGetter keyGetter,
+            ArrayData.ElementGetter valueGetter) {
+        return new MapUnnestWithOrdinalityFunction(
+                context, wrapWithOrdinality(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 RowData.FieldGetter[] fieldGetters = null;
+
+        public CollectionUnnestWithOrdinalityFunction(
+                SpecializedContext context,
+                LogicalType elementType,
+                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) {
+            evalArrayData(arrayData, elementGetter, 
this::collectWithOrdinality);
+        }
+
+        public void eval(MapData mapData) {
+            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);

Review Comment:
   We should be able to use `JoinedRowData` here. This avoid field getters as 
the original row can be passed.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/UnnestRowsFunctionBase.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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;
+
+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 {
+
+    public UnnestRowsFunctionBase(boolean withOrdinality) {
+        super(
+                withOrdinality
+                        ? 
BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS_WITH_ORDINALITY
+                        : 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 elementType;
+        switch (logicalType.getTypeRoot()) {
+            case ARRAY:
+                elementType = ((ArrayType) logicalType).getElementType();
+                break;
+            case MULTISET:
+                elementType = ((MultisetType) logicalType).getElementType();
+                break;
+            case MAP:
+                MapType mapType = (MapType) logicalType;
+                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;
+            return new RowType(
+                    false,
+                    Stream.concat(
+                                    rowType.getFields().stream(),
+                                    Stream.of(
+                                            new RowType.RowField(
+                                                    "ordinality",

Review Comment:
   ```suggestion
                                                       "ORDINALITY",
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to