liuyongvs commented on a change in pull request #12964: URL: https://github.com/apache/flink/pull/12964#discussion_r461479018
########## File path: flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.xml ########## @@ -0,0 +1,171 @@ +<?xml version="1.0" ?> +<!-- +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. +--> +<Root> + <TestCase name="testCanPushdownLimitWithoutOffset"> + <Resource name="sql"> + <![CDATA[SELECT * FROM LimitTable LIMIT 5]]> + </Resource> + <Resource name="planBefore"> + <![CDATA[ +LogicalSort(fetch=[5]) ++- LogicalProject(a=[$0], b=[$1], c=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]]) +]]> + </Resource> + <Resource name="planAfter"> + <![CDATA[ +LogicalProject(a=[$0], b=[$1], c=[$2]) ++- FlinkLogicalSort(fetch=[5]) + +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 5]]], fields=[a, b, c]) Review comment: This is PushLimitInto**Legacy**TableSourceScanRuleTest, which is legacy Rule.And read the code of TestLegacyLimitableTableSource.explainSource. You can compare the PushProjectIntoLegacyTableSourceScanRuleTest and PushProjectIntoTableSourceScanRuleTest. And there are the same. new: LogicalTableScan(table=[[default_catalog, default_database, MyTable, project=[a, c]]]) Legacy: LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestSource(physical fields: a, c)]]]) ########## File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.scala ########## @@ -0,0 +1,116 @@ +/* + * 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.calcite.plan.hep.HepMatchOrder +import org.apache.calcite.rel.rules.SortProjectTransposeRule +import org.apache.calcite.tools.RuleSets +import org.apache.flink.table.api.SqlParserException +import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalLegacyTableSourceScan, FlinkLogicalSort} +import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE} +import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} +import org.junit.{Before, Test} + +/** + * Test for [[PushLimitIntoLegacyTableSourceScanRule]]. + */ +class PushLimitIntoLegacyTableSourceScanRuleTest extends TableTestBase { + protected val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE) + val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig) + calciteConfig.getBatchProgram.get.addLast( + "rules", + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) + .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) + .add(RuleSets.ofList(PushLimitIntoLegacyTableSourceScanRule.INSTANCE, + SortProjectTransposeRule.INSTANCE, + // converts calcite rel(RelNode) to flink rel(FlinkRelNode) + FlinkLogicalSort.BATCH_CONVERTER, + FlinkLogicalLegacyTableSourceScan.CONVERTER)) + .build() + ) + + val ddl = + s""" + |CREATE TABLE LimitTable ( + | a int, + | b bigint, + | c string + |) WITH ( + | 'connector.type' = 'TestLimitableTableSource', + | 'is-bounded' = 'true' + |) + """.stripMargin + util.tableEnv.executeSql(ddl) + } + + @Test(expected = classOf[SqlParserException]) + def testLimitWithNegativeOffset(): Unit = { + util.verifyPlan("SELECT a, c FROM LimitTable LIMIT 10 OFFSET -1") + } + + @Test(expected = classOf[SqlParserException]) + def testNegativeLimitWithoutOffset(): Unit = { + util.verifyPlan("SELECT * FROM LimitTable LIMIT -1") + } + + @Test(expected = classOf[SqlParserException]) + def testMysqlLimit(): Unit = { + util.verifyPlan("SELECT a, c FROM LimitTable LIMIT 1, 10") + } + + @Test + def testCanPushdownLimitWithoutOffset(): Unit = { + util.verifyPlan("SELECT * FROM LimitTable LIMIT 5") + } + + @Test + def testCanPushdownLimitWithOffset(): Unit = { + util.verifyPlan("SELECT a, c FROM LimitTable LIMIT 10 OFFSET 1") + } + + @Test + def testCanPushdownFetchWithOffset(): Unit = { + util.verifyPlan("SELECT a, c FROM LimitTable OFFSET 10 ROWS FETCH NEXT 10 ROWS ONLY") + } + + @Test + def testCanPushdownFetchWithoutOffset(): Unit = { + util.verifyPlan("SELECT a, c FROM LimitTable FETCH FIRST 10 ROWS ONLY") + } + + @Test + def testCannotPushDownWithoutLimit(): Unit = { Review comment: ok ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java ########## @@ -0,0 +1,122 @@ +/* + * 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.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalSort; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan; +import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rex.RexLiteral; + +import java.util.Arrays; +import java.util.Collections; + +/** + * Planner rule that tries to push limit into a {@link FlinkLogicalTableSourceScan}, + * which table is a {@link TableSourceTable}. And the table source in the table is a {@link SupportsLimitPushDown}. + * The original limit will still be retained. + * The reasons why the limit still be retained: + * 1.If the source is required to return the exact number of limit number, the implementation + * of the source is highly required. The source is required to accurately control the record + * number of split, and the parallelism setting also need to be adjusted accordingly. + * 2.When remove the limit, maybe filter will be pushed down to the source after limit pushed + * down. The source need know it should do limit first and do the filter later, it is hard to + * implement. + * 3.We can support limit with offset, we can push down offset + fetch to table source. + */ +public class PushLimitIntoTableSourceScanRule extends RelOptRule { + public static final PushLimitIntoTableSourceScanRule INSTANCE = new PushLimitIntoTableSourceScanRule(); + + public PushLimitIntoTableSourceScanRule() { + super(operand(FlinkLogicalSort.class, + operand(FlinkLogicalTableSourceScan.class, none())), + "PushLimitIntoTableSourceScanRule"); + } + + @Override + public boolean matches(RelOptRuleCall call) { + Sort sort = call.rel(0); + TableSourceTable tableSourceTable = call.rel(1).getTable().unwrap(TableSourceTable.class); + + // a limit can be pushed down only if it satisfies the two conditions: 1) do not have order by keys, 2) have limit. + boolean onlyLimit = sort.getCollation().getFieldCollations().isEmpty() && sort.fetch != null; + return onlyLimit + && tableSourceTable != null + && tableSourceTable.tableSource() instanceof SupportsLimitPushDown + && Arrays.stream(tableSourceTable.extraDigests()).noneMatch(str -> str.startsWith("limit=[")); + } + + @Override + public void onMatch(RelOptRuleCall call) { + Sort sort = call.rel(0); + FlinkLogicalTableSourceScan scan = call.rel(1); + TableSourceTable tableSourceTable = scan.getTable().unwrap(TableSourceTable.class); + int offset = sort.offset == null ? 0 : RexLiteral.intValue(sort.offset); + int limit = offset + RexLiteral.intValue(sort.fetch); + + TableSourceTable newTableSourceTable = applyLimit(limit, tableSourceTable); + + FlinkLogicalTableSourceScan newScan = FlinkLogicalTableSourceScan.create(scan.getCluster(), newTableSourceTable); + Sort newSort = sort.copy(sort.getTraitSet(), Collections.singletonList(newScan)); + call.transformTo(newSort); + } + + private TableSourceTable applyLimit( + long limit, + FlinkPreparingTableBase relOptTable) { + TableSourceTable oldTableSourceTable = relOptTable.unwrap(TableSourceTable.class); + DynamicTableSource newTableSource = oldTableSourceTable.tableSource().copy(); + ((SupportsLimitPushDown) newTableSource).applyLimit(limit); + + FlinkStatistic statistic = relOptTable.getStatistic(); + long newRowCount = 0; + if (statistic.getRowCount() != null) { + newRowCount = Math.min(limit, statistic.getRowCount().longValue()); + } else { + newRowCount = limit; + } + // update TableStats after limit push down + TableStats newTableStats = new TableStats(newRowCount); + FlinkStatistic newStatistic = FlinkStatistic.builder() + .statistic(statistic) + .tableStats(newTableStats) + .build(); + + // update extraDigests + String[] newExtraDigests = new String[0]; + if (limit >= 0) { Review comment: I think it can not remove it. Although, it doesn't have problem now. But if the calcite supports this syntax limit x,y, which is the mysql limit offset syntax. y represents limit, which can be negative, such as -1 , you can refer this https://www.cnblogs.com/acm-bingzi/p/msqlLimit.html. And you can also read the code of calcite FlinkSqlParserImpl.OrderedQueryOrExpr, where has some comments. That is the reason why i add this test testMysqlLimit. ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java ########## @@ -0,0 +1,122 @@ +/* + * 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.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalSort; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan; +import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rex.RexLiteral; + +import java.util.Arrays; +import java.util.Collections; + +/** + * Planner rule that tries to push limit into a {@link FlinkLogicalTableSourceScan}, + * which table is a {@link TableSourceTable}. And the table source in the table is a {@link SupportsLimitPushDown}. + * The original limit will still be retained. + * The reasons why the limit still be retained: + * 1.If the source is required to return the exact number of limit number, the implementation + * of the source is highly required. The source is required to accurately control the record + * number of split, and the parallelism setting also need to be adjusted accordingly. + * 2.When remove the limit, maybe filter will be pushed down to the source after limit pushed + * down. The source need know it should do limit first and do the filter later, it is hard to + * implement. + * 3.We can support limit with offset, we can push down offset + fetch to table source. + */ +public class PushLimitIntoTableSourceScanRule extends RelOptRule { + public static final PushLimitIntoTableSourceScanRule INSTANCE = new PushLimitIntoTableSourceScanRule(); + + public PushLimitIntoTableSourceScanRule() { + super(operand(FlinkLogicalSort.class, + operand(FlinkLogicalTableSourceScan.class, none())), + "PushLimitIntoTableSourceScanRule"); + } + + @Override + public boolean matches(RelOptRuleCall call) { + Sort sort = call.rel(0); + TableSourceTable tableSourceTable = call.rel(1).getTable().unwrap(TableSourceTable.class); + + // a limit can be pushed down only if it satisfies the two conditions: 1) do not have order by keys, 2) have limit. + boolean onlyLimit = sort.getCollation().getFieldCollations().isEmpty() && sort.fetch != null; + return onlyLimit + && tableSourceTable != null + && tableSourceTable.tableSource() instanceof SupportsLimitPushDown + && Arrays.stream(tableSourceTable.extraDigests()).noneMatch(str -> str.startsWith("limit=[")); + } + + @Override + public void onMatch(RelOptRuleCall call) { + Sort sort = call.rel(0); + FlinkLogicalTableSourceScan scan = call.rel(1); + TableSourceTable tableSourceTable = scan.getTable().unwrap(TableSourceTable.class); + int offset = sort.offset == null ? 0 : RexLiteral.intValue(sort.offset); + int limit = offset + RexLiteral.intValue(sort.fetch); + + TableSourceTable newTableSourceTable = applyLimit(limit, tableSourceTable); + + FlinkLogicalTableSourceScan newScan = FlinkLogicalTableSourceScan.create(scan.getCluster(), newTableSourceTable); + Sort newSort = sort.copy(sort.getTraitSet(), Collections.singletonList(newScan)); + call.transformTo(newSort); + } + + private TableSourceTable applyLimit( + long limit, + FlinkPreparingTableBase relOptTable) { + TableSourceTable oldTableSourceTable = relOptTable.unwrap(TableSourceTable.class); + DynamicTableSource newTableSource = oldTableSourceTable.tableSource().copy(); + ((SupportsLimitPushDown) newTableSource).applyLimit(limit); + + FlinkStatistic statistic = relOptTable.getStatistic(); + long newRowCount = 0; + if (statistic.getRowCount() != null) { + newRowCount = Math.min(limit, statistic.getRowCount().longValue()); + } else { + newRowCount = limit; + } + // update TableStats after limit push down + TableStats newTableStats = new TableStats(newRowCount); + FlinkStatistic newStatistic = FlinkStatistic.builder() + .statistic(statistic) + .tableStats(newTableStats) + .build(); + + // update extraDigests + String[] newExtraDigests = new String[0]; + if (limit >= 0) { Review comment: I think it can not be removed. Although, it doesn't have problem now. But if the calcite supports this syntax limit x,y, which is the mysql limit offset syntax. y represents limit, which can be negative, such as -1 , you can refer this https://www.cnblogs.com/acm-bingzi/p/msqlLimit.html. And you can also read the code of calcite FlinkSqlParserImpl.OrderedQueryOrExpr, where has some comments. That is the reason why i add this test testMysqlLimit. ########## File path: flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRuleTest.xml ########## @@ -0,0 +1,171 @@ +<?xml version="1.0" ?> +<!-- +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. +--> +<Root> + <TestCase name="testCanPushdownLimitWithoutOffset"> + <Resource name="sql"> + <![CDATA[SELECT * FROM LimitTable LIMIT 5]]> + </Resource> + <Resource name="planBefore"> + <![CDATA[ +LogicalSort(fetch=[5]) ++- LogicalProject(a=[$0], b=[$1], c=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]]) +]]> + </Resource> + <Resource name="planAfter"> + <![CDATA[ +LogicalProject(a=[$0], b=[$1], c=[$2]) ++- FlinkLogicalSort(fetch=[5]) + +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 5]]], fields=[a, b, c]) Review comment: This is PushLimitInto**Legacy**TableSourceScanRuleTest, which is legacy Rule.And read the code of TestLegacyLimitableTableSource.explainSource. You can compare the PushProjectIntoLegacyTableSourceScanRuleTest and PushProjectIntoTableSourceScanRuleTest. And there are the same. **new:** LogicalTableScan(table=[[default_catalog, default_database, MyTable, project=[a, c]]]) **Legacy:** LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestSource(physical fields: a, c)]]]) ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
