liuyongvs commented on a change in pull request #12964:
URL: https://github.com/apache/flink/pull/12964#discussion_r462766978



##########
File path: 
flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRuleTest.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 a, c FROM LimitTable LIMIT 5]]>
+               </Resource>
+               <Resource name="planBefore">
+                       <![CDATA[
+LogicalSort(fetch=[5])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]])
+]]>
+               </Resource>
+               <Resource name="planAfter">
+                       <![CDATA[
+LogicalProject(a=[$0], c=[$2])
++- FlinkLogicalSort(fetch=[5])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable, limit=[5]]], fields=[a, b, c])
+]]>
+               </Resource>
+       </TestCase>
+       <TestCase name="testCanPushdownFetchWithOffset">
+               <Resource name="sql">
+                       <![CDATA[SELECT a, c FROM LimitTable OFFSET 10 ROWS 
FETCH NEXT 10 ROWS ONLY]]>
+               </Resource>
+               <Resource name="planBefore">
+                       <![CDATA[
+LogicalSort(offset=[10], fetch=[10])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]])
+]]>
+               </Resource>
+               <Resource name="planAfter">
+                       <![CDATA[
+LogicalProject(a=[$0], c=[$2])
++- FlinkLogicalSort(offset=[10], fetch=[10])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable, limit=[20]]], fields=[a, b, c])
+]]>
+               </Resource>
+       </TestCase>
+       <TestCase name="testCanPushdownFetchWithoutOffset">
+               <Resource name="sql">
+                       <![CDATA[SELECT a, c FROM LimitTable FETCH FIRST 10 
ROWS ONLY]]>
+               </Resource>
+               <Resource name="planBefore">
+                       <![CDATA[
+LogicalSort(fetch=[10])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]])
+]]>
+               </Resource>
+               <Resource name="planAfter">
+                       <![CDATA[
+LogicalProject(a=[$0], c=[$2])
++- FlinkLogicalSort(fetch=[10])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable, limit=[10]]], fields=[a, b, c])
+]]>
+               </Resource>
+       </TestCase>
+       <TestCase name="testCannotPushDownWithOrderBy">
+               <Resource name="sql">
+                       <![CDATA[SELECT a, c FROM LimitTable ORDER BY c LIMIT 
10]]>
+               </Resource>
+               <Resource name="planBefore">
+                       <![CDATA[
+LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]])
+]]>
+               </Resource>
+               <Resource name="planAfter">
+                       <![CDATA[
+LogicalProject(a=[$0], c=[$2])
++- FlinkLogicalSort(sort0=[$2], dir0=[ASC-nulls-first], fetch=[10])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable]], fields=[a, b, c])
+]]>
+               </Resource>
+       </TestCase>
+       <TestCase name="testCanPushdownLimitWithOffset">
+               <Resource name="sql">
+                       <![CDATA[SELECT a, c FROM LimitTable LIMIT 10 OFFSET 
1]]>
+               </Resource>
+               <Resource name="planBefore">
+                       <![CDATA[
+LogicalSort(offset=[1], fetch=[10])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[default_catalog, default_database, LimitTable]])
+]]>
+               </Resource>
+               <Resource name="planAfter">
+                       <![CDATA[
+LogicalProject(a=[$0], c=[$2])
++- FlinkLogicalSort(offset=[1], fetch=[10])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable, limit=[11]]], fields=[a, b, c])
+]]>
+               </Resource>
+       </TestCase>
+       <TestCase name="testLimitWithoutOffset">
+               <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])
+   +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, 
LimitTable, source: [limit: 5]]], fields=[a, b, c])

Review comment:
       sorry, it should be removed. i have replace the testLimitWithoutOffset 
by  testCanPushdownLimitWithoutOffset. And forgot to remove it. 

##########
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:
       good idea

##########
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:
       good idea

##########
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:
       Yeap, calcite fetch member is null when it doesn't have limit currently. 
And calcite doesn't support mysql  limit x,y syntax.  If it supports, limit 5, 
-1. The -1 represents the end. That is to say [offset5, end).
   So the calcite may transform -1 to fetch null when it supports in parser. 
And we don't need to limit >=0.
   So i just do as you say, remove the limit >= 0 and  don't add limitation in 
matches.




----------------------------------------------------------------
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]


Reply via email to