[ 
https://issues.apache.org/jira/browse/FLINK-9713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16619322#comment-16619322
 ] 

ASF GitHub Bot commented on FLINK-9713:
---------------------------------------

pnowojski commented on a change in pull request #6299: [FLINK-9713][table][sql] 
Support processing time versioned joins
URL: https://github.com/apache/flink/pull/6299#discussion_r218439267
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToTemporalTableJoinRule.scala
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * 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.logical
+
+import org.apache.calcite.plan.RelOptRule.{any, none, operand, some}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.TableFunctionScan
+import org.apache.calcite.rel.logical.LogicalCorrelate
+import org.apache.calcite.rex._
+import org.apache.flink.table.api.{Table, Types, ValidationException}
+import org.apache.flink.table.calcite.FlinkTypeFactory.isProctimeIndicatorType
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.functions.TemporalTableFunction
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin
+import org.apache.flink.table.plan.util.RexDefaultVisitor
+
+class LogicalCorrelateToTemporalTableJoinRule
+  extends RelOptRule(
+    operand(classOf[LogicalCorrelate],
+      some(
+        operand(classOf[RelNode], any()),
+        operand(classOf[TableFunctionScan], none()))),
+    "LogicalCorrelateToTemporalTableJoinRule") {
+
+  def extractNameFromTimeAttribute(timeAttribute: Expression): String = {
+    timeAttribute match {
+      case RowtimeAttribute(expr: TemporalTableReference) =>
+        expr.name
+      case ProctimeAttribute(expr: TemporalTableReference) =>
+        expr.name
+      case TemporalTableReference(name, _) =>
+        name
+      case ResolvedFieldReference(name, _)
+        if timeAttribute.resultType == Types.LONG ||
+          timeAttribute.resultType == Types.SQL_TIMESTAMP =>
+        name
+      case _ => throw new ValidationException(
+        s"Invalid timeAttribute [${timeAttribute}] in TemporalTableFunction")
+    }
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val logicalCorrelate: LogicalCorrelate = call.rel(0)
+    val leftNode: RelNode = call.rel(1)
+    val rightTableFunctionScan: TableFunctionScan = call.rel(2)
+
+    val cluster = logicalCorrelate.getCluster
+
+    new GetTemporalTableFunctionCall(cluster.getRexBuilder, leftNode)
+      .visit(rightTableFunctionScan.getCall) match {
+      case Some(TemporalTableFunctionCall(rightTemporalTableFunction, 
leftTimeAttribute)) =>
+        val underlyingHistoryTable: Table = 
rightTemporalTableFunction.getUnderlyingHistoryTable
+        val relBuilder = this.relBuilderFactory.create(
+          cluster,
+          underlyingHistoryTable.relBuilder.getRelOptSchema)
+        val rexBuilder = cluster.getRexBuilder
+
+        val rightNode: RelNode = 
underlyingHistoryTable.logicalPlan.toRelNode(relBuilder)
+
+        val rightTimeIndicatorExpression = createRightExpression(
+          rexBuilder,
+          leftNode,
+          rightNode,
+          
extractNameFromTimeAttribute(rightTemporalTableFunction.getTimeAttribute))
+
+        val rightPrimaryKeyExpression = createRightExpression(
+          rexBuilder,
+          leftNode,
+          rightNode,
+          rightTemporalTableFunction.getPrimaryKey)
+
+        relBuilder.push(
+          if 
(isProctimeIndicatorType(rightTemporalTableFunction.getTimeAttribute.resultType))
 {
+            LogicalTemporalTableJoin.create(
+              rexBuilder,
+              cluster,
+              logicalCorrelate.getTraitSet,
+              leftNode,
+              rightNode,
+              leftTimeAttribute,
+              rightPrimaryKeyExpression)
+          }
+          else {
+            LogicalTemporalTableJoin.create(
+              rexBuilder,
+              cluster,
+              logicalCorrelate.getTraitSet,
+              leftNode,
+              rightNode,
+              leftTimeAttribute,
+              rightTimeIndicatorExpression,
+              rightPrimaryKeyExpression)
+          })
+        call.transformTo(relBuilder.build())
+      case None =>
+    }
+  }
+
+  private def createRightExpression(
+      rexBuilder: RexBuilder,
+      leftNode: RelNode,
+      rightNode: RelNode,
+      field: String): RexNode = {
+    val rightReferencesOffset = leftNode.getRowType.getFieldCount
+    val rightDataTypeField = rightNode.getRowType.getField(field, false, false)
+    rexBuilder.makeInputRef(
+      rightDataTypeField.getType, rightReferencesOffset + 
rightDataTypeField.getIndex)
+  }
+}
+
+object LogicalCorrelateToTemporalTableJoinRule {
+  val INSTANCE: RelOptRule = new LogicalCorrelateToTemporalTableJoinRule
+}
+
+/**
+  * Simple pojo class for extracted [[TemporalTableFunction]] with time 
attribute
+  * extracted from RexNode with [[TemporalTableFunction]] call.
+  */
+case class TemporalTableFunctionCall(
+    var temporalTableFunction: TemporalTableFunction,
+    var timeAttribute: RexNode) {
+}
+
+/**
+  * Find [[TemporalTableFunction]] call and run 
[[CorrelatedFieldAccessRemoval]] on it's operand.
+  */
+class GetTemporalTableFunctionCall(
+    var rexBuilder: RexBuilder,
+    var leftSide: RelNode)
+  extends RexVisitorImpl[TemporalTableFunctionCall](false) {
+
+  def visit(node: RexNode): Option[TemporalTableFunctionCall] = {
+    val result = node.accept(this)
+    if (result == null) {
+      return None
+    }
+    Some(result)
+  }
+
+  override def visitCall(rexCall: RexCall): TemporalTableFunctionCall = {
+    if (!rexCall.getOperator.isInstanceOf[TableSqlFunction]) {
+      return null
 
 Review comment:
   I wanted to, but `null` is enforced by `RexVisitorImpl` :( So we are stuck 
with it unless we want to replace it with our own version.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> Support versioned joins in planning phase
> -----------------------------------------
>
>                 Key: FLINK-9713
>                 URL: https://issues.apache.org/jira/browse/FLINK-9713
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>    Affects Versions: 1.5.0
>            Reporter: Piotr Nowojski
>            Assignee: Piotr Nowojski
>            Priority: Major
>              Labels: pull-request-available
>
> Queries like:
> {code:java}
> SELECT 
>   o.amount * r.rate 
> FROM 
>   Orders AS o, 
>   LATERAL TABLE (Rates(o.rowtime)) AS r 
> WHERE o.currency = r.currency{code}
> should evaluate to valid plan with versioned joins plan node.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to