KurtYoung commented on a change in pull request #8302: 
[FLINK-12269][table-blink] Support Temporal Table Join in blink planner and 
runtime
URL: https://github.com/apache/flink/pull/8302#discussion_r281520071
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/common/CommonTemporalTableJoinRule.scala
 ##########
 @@ -0,0 +1,134 @@
+/*
+ * 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.plan.rules.physical.common
+
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rex.{RexNode, RexProgram}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.nodes.common.CommonTemporalTableJoin
+import org.apache.flink.table.plan.nodes.logical._
+import org.apache.flink.table.plan.nodes.physical.PhysicalTableSourceScan
+import org.apache.flink.table.sources.{LookupableTableSource, TableSource}
+
+/**
+  * Base implementation for both
+  * 
[[org.apache.flink.table.plan.nodes.physical.batch.BatchExecTemporalTableJoin]] 
and
+  * 
[[org.apache.flink.table.plan.nodes.physical.stream.StreamExecTemporalTableJoin]].
+  */
+trait CommonTemporalTableJoinRule {
+
+  protected def matches(join: FlinkLogicalJoin, tableScan: TableScan): Boolean 
= {
+    // TODO: shouldn't match temporal table function join
+    if (findTableSource(tableScan).isEmpty) {
+      throw new TableException(
+        "Temporal table join only support join on a LookupableTableSource " +
+          "not on a DataStream or an intermediate query")
+    }
+    // currently temporal table join only support LookupableTableSource
+    isLookupableTableSource(tableScan)
+  }
+
+  protected def findTableSource(relNode: RelNode): Option[TableSource[_]] = {
+    relNode match {
+      case logicalScan: FlinkLogicalTableSourceScan => 
Some(logicalScan.tableSource)
+      case physicalScan: PhysicalTableSourceScan => 
Some(physicalScan.tableSource)
+      // TODO: find TableSource in FlinkLogicalIntermediateTableScan
+      case _ => None
+    }
+  }
+
+  protected def isLookupableTableSource(relNode: RelNode): Boolean = {
+    relNode match {
+      case logicalScan: FlinkLogicalTableSourceScan =>
+        logicalScan.tableSource.isInstanceOf[LookupableTableSource[_]]
+      case physicalScan: PhysicalTableSourceScan =>
+        physicalScan.tableSource.isInstanceOf[LookupableTableSource[_]]
+      // TODO: find TableSource in FlinkLogicalIntermediateTableScan
+      case _ => false
+    }
+  }
+
+  protected def transform(
+    join: FlinkLogicalJoin,
+    input: FlinkLogicalRel,
+    tableSource: TableSource[_],
+    period: RexNode,
+    calcProgram: Option[RexProgram]): CommonTemporalTableJoin
+}
+
+abstract class BaseSnapshotOnTableScanRule(description: String)
+  extends RelOptRule(
+    operand(classOf[FlinkLogicalJoin],
+      operand(classOf[FlinkLogicalRel], any()),
+      operand(classOf[FlinkLogicalSnapshot],
+        operand(classOf[TableScan], any()))),
+    description)
+          with CommonTemporalTableJoinRule {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val join = call.rel[FlinkLogicalJoin](0)
+    val tableScan = call.rel[TableScan](3)
+    matches(join, tableScan)
 
 Review comment:
   I noticed current TemporalTableJoin is a `SingleRel`, which is not suitable 
for further extension after we support scanning data into state and provide 
event time join. So i think it's inappropriate to translate snapshotted with 
rowtime to `TemporalTableJoin` and then throw exception inside this physical 
operator.

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