wuchong commented on a change in pull request #9876: [FLINK-14134][table] 
Introduce LimitableTableSource for optimizing limit
URL: https://github.com/apache/flink/pull/9876#discussion_r339395422
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala
 ##########
 @@ -0,0 +1,127 @@
+/*
+ * 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.api.TableException
+import org.apache.flink.table.plan.stats.TableStats
+import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalSort, 
FlinkLogicalTableSourceScan}
+import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, 
TableSourceTable}
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic
+import org.apache.flink.table.sources.LimitableTableSource
+
+import org.apache.calcite.plan.RelOptRule.{none, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.core.{Sort, TableScan}
+import org.apache.calcite.rex.RexLiteral
+import org.apache.calcite.tools.RelBuilder
+
+import java.util.Collections
+
+/**
+  * Planner rule that tries to push limit into a [[LimitableTableSource]].
+  * 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.
+  */
+class PushLimitIntoTableSourceScanRule extends RelOptRule(
+  operand(classOf[FlinkLogicalSort],
+    operand(classOf[FlinkLogicalTableSourceScan], none)), 
"PushLimitIntoTableSourceScanRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val sort = call.rel(0).asInstanceOf[Sort]
+    val fetch = sort.fetch
+    val offset = sort.offset
+    // Only push-down the limit whose offset equal zero. Because it is 
difficult to source based
+    // push to handle the non-zero offset. And the non-zero offset usually 
appear together with
+    // sort.
+    val onlyLimit = sort.getCollation.getFieldCollations.isEmpty &&
+        (offset == null || RexLiteral.intValue(offset) == 0) &&
+        fetch != null
+
+    var supportPushDown = false
+    if (onlyLimit) {
+      supportPushDown = call.rel(1).asInstanceOf[TableScan]
+          .getTable.unwrap(classOf[TableSourceTable[_]]) match {
+        case table: TableSourceTable[_] =>
+          table.tableSource match {
+            case source: LimitableTableSource[_] => !source.isLimitPushedDown
+            case _ => false
+          }
+        case _ => false
+      }
+    }
+    supportPushDown
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val sort = call.rel(0).asInstanceOf[Sort]
+    val scan = call.rel(1).asInstanceOf[FlinkLogicalTableSourceScan]
+    val relOptTable = scan.getTable.asInstanceOf[FlinkRelOptTable]
+    val limit = RexLiteral.intValue(sort.fetch)
+    val relBuilder = call.builder()
+    val newRelOptTable = applyLimit(limit, relOptTable, relBuilder)
+    val newScan = scan.copy(scan.getTraitSet, newRelOptTable)
+
+    val newTableSource = 
newRelOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource
+    val oldTableSource = 
relOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource
+
+    if (newTableSource.asInstanceOf[LimitableTableSource[_]].isLimitPushedDown
+        && 
newTableSource.explainSource().equals(oldTableSource.explainSource)) {
+      throw new TableException("Failed to push limit into table source! "
 
 Review comment:
   Actually, I have some concerns on this. I also noticed we did the similar 
things in pushing filter and pushing projection rules. However, I think it 
pushes some dirty work to connectos, makes connectors more difficult to develop 
**correctly**. IMO, the digest can be generated by planner correctly somehow. 

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to