LadyForest commented on code in PR #24510: URL: https://github.com/apache/flink/pull/24510#discussion_r1568258000
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/UncollectToTableFunctionScanRule.java: ########## @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.rules.logical; + +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +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.types.logical.LogicalType; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Uncollect; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.logical.LogicalTableFunctionScan; +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; + +import static org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType; + +/** + * Planner rule that converts [[Uncollect]] values to + * [[org.apache.calcite.rel.core.TableFunctionScan]]. + */ Review Comment: Nit: replace with java doc link ```suggestion /** * Planner rule that converts {@link Uncollect} values to {@link * org.apache.calcite.rel.core.TableFunctionScan}. */ ``` ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.scala: ########## @@ -19,8 +19,34 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.table.planner.plan.common.UnnestTestBase import org.apache.flink.table.planner.utils.TableTestUtil +import org.apache.flink.types.Row +import org.apache.flink.util.CollectionUtil + +import org.assertj.core.api.Assertions.assertThat +import org.junit.jupiter.api.Test class UnnestTest extends UnnestTestBase(true) { override def getTableTestUtil: TableTestUtil = batchTestUtil() + + @Test + def testUnnestWithValuesBatch(): Unit = { + val src = util.tableEnv.sqlQuery("SELECT * FROM UNNEST(ARRAY[1,2,3])") + val rows: java.util.List[Row] = CollectionUtil.iteratorToList(src.execute.collect) + assertThat(rows.size()).isEqualTo(3) + assertThat(rows.get(0).toString).isEqualTo("+I[1]") + assertThat(rows.get(1).toString).isEqualTo("+I[2]") + assertThat(rows.get(2).toString).isEqualTo("+I[3]") + } + + @Test + def testUnnestWithValuesBatch2(): Unit = { + val src = + util.tableEnv.sqlQuery("SELECT * FROM (VALUES('a')) CROSS JOIN UNNEST(ARRAY[1, 2, 3])") + val rows: java.util.List[Row] = CollectionUtil.iteratorToList(src.execute.collect) + assertThat(rows.size()).isEqualTo(3) + assertThat(rows.get(0).toString).isEqualTo("+I[a, 1]") + assertThat(rows.get(1).toString).isEqualTo("+I[a, 2]") + assertThat(rows.get(2).toString).isEqualTo("+I[a, 3]") + } Review Comment: `org.apache.flink.table.planner.plan.batch.sql.UnnestTest` and `org.apache.flink.table.planner.plan.stream.sql.UnnestTest` are mainly focusing on the plan test. Would you mind moving these tests to `org.apache.flink.table.planner.runtime.batch.sql.UnnestITCase` and `org.apache.flink.table.planner.runtime.stream.sql.UnnestITCase` respectively? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
