JingsongLi commented on a change in pull request #8901: 
[FLINK-13003][table-planner-blink] Support Temporal TableFunction Join in blink 
planner
URL: https://github.com/apache/flink/pull/8901#discussion_r298441480
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalTableJoin.scala
 ##########
 @@ -0,0 +1,422 @@
+/*
+ * 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.nodes.physical.stream
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator
+import org.apache.flink.streaming.api.transformations.{StreamTransformation, 
TwoInputTransformation}
+import org.apache.flink.table.api.{StreamTableEnvironment, TableConfig, 
TableException, ValidationException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import 
org.apache.flink.table.calcite.FlinkTypeFactory.{isProctimeIndicatorType, 
isRowtimeIndicatorType}
+import org.apache.flink.table.codegen.{CodeGeneratorContext, 
ExprCodeGenerator, FunctionCodeGenerator}
+import org.apache.flink.table.dataformat.BaseRow
+import org.apache.flink.table.generated.GeneratedJoinCondition
+import org.apache.flink.table.plan.nodes.common.CommonPhysicalJoin
+import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
+import 
org.apache.flink.table.plan.util.TemporalJoinUtil.TEMPORAL_JOIN_CONDITION
+import org.apache.flink.table.plan.util.{InputRefVisitor, KeySelectorUtil, 
RelExplainUtil, TemporalJoinUtil}
+import 
org.apache.flink.table.runtime.join.temporal.{TemporalProcessTimeJoinOperator, 
TemporalRowTimeJoinOperator}
+import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector
+import org.apache.flink.table.types.logical.RowType
+import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.util.Preconditions.checkState
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.{Join, JoinInfo, JoinRelType}
+import org.apache.calcite.rex._
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+class StreamExecTemporalTableJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftRel: RelNode,
+    rightRel: RelNode,
+    condition: RexNode,
+    joinType: JoinRelType)
+  extends CommonPhysicalJoin(cluster, traitSet, leftRel, rightRel, condition, 
joinType)
+  with StreamPhysicalRel
+  with StreamExecNode[BaseRow] {
+
+  override def producesUpdates: Boolean = false
+
+  override def needsUpdatesAsRetraction(input: RelNode): Boolean = false
+
+  override def consumesRetractions: Boolean = false
+
+  override def producesRetractions: Boolean = false
+
+  override def requireWatermark: Boolean = {
+    val nonEquiJoinRex = getJoinInfo.getRemaining(cluster.getRexBuilder)
+
+    var rowtimeJoin: Boolean = false
+    val visitor = new RexVisitorImpl[Unit](true) {
+      override def visitCall(call: RexCall): Unit = {
+        if (call.getOperator == TEMPORAL_JOIN_CONDITION) {
+          rowtimeJoin = TemporalJoinUtil.isRowtimeCall(call)
+        } else {
+          call.getOperands.foreach(node => node.accept(this))
+        }
+      }
+    }
+    nonEquiJoinRex.accept(visitor)
+    rowtimeJoin
+  }
+
+  override def copy(
+      traitSet: RelTraitSet,
+      conditionExpr: RexNode,
+      left: RelNode,
+      right: RelNode,
+      joinType: JoinRelType,
+      semiJoinDone: Boolean): Join = {
+    new StreamExecTemporalTableJoin(
+      cluster,
+      traitSet,
+      left,
+      right,
+      conditionExpr,
+      joinType)
+  }
+
+  //~ ExecNode methods 
-----------------------------------------------------------
+
+  override def getInputNodes: util.List[ExecNode[StreamTableEnvironment, _]] = 
{
+    getInputs.map(_.asInstanceOf[ExecNode[StreamTableEnvironment, _]])
+  }
+
+  override def replaceInputNode(
+    ordinalInParent: Int,
+    newInputNode: ExecNode[StreamTableEnvironment, _]): Unit = {
+    replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode])
+  }
+
+  override protected def translateToPlanInternal(
+    tableEnv: StreamTableEnvironment): StreamTransformation[BaseRow] = {
+
+    validateKeyTypes()
+
+    val returnType = FlinkTypeFactory.toLogicalRowType(getRowType)
+
+    val joinTranslator = StreamExecTemporalJoinToCoProcessTranslator.create(
+      this.toString,
+      tableEnv.getConfig,
+      returnType,
+      leftRel,
+      rightRel,
+      getJoinInfo,
+      cluster.getRexBuilder)
+
+    val joinOperator = joinTranslator.getJoinOperator(joinType, 
returnType.getFieldNames)
+    val leftKeySelector = joinTranslator.getLeftKeySelector
+    val rightKeySelector = joinTranslator.getRightKeySelector
+
+    val leftTransform = getInputNodes.get(0).translateToPlan(tableEnv)
+      .asInstanceOf[StreamTransformation[BaseRow]]
+    val rightTransform = getInputNodes.get(1).translateToPlan(tableEnv)
+      .asInstanceOf[StreamTransformation[BaseRow]]
+
+    val ret = new TwoInputTransformation[BaseRow, BaseRow, BaseRow](
+      leftTransform,
+      rightTransform,
+      getJoinOperatorName,
+      joinOperator,
+      BaseRowTypeInfo.of(returnType),
+      getResource.getParallelism)
+
+    if (getResource.getMaxParallelism > 0) {
+      ret.setMaxParallelism(getResource.getMaxParallelism)
+    }
+
+    // set KeyType and Selector for state
+    ret.setStateKeySelectors(leftKeySelector, rightKeySelector)
+    
ret.setStateKeyType(leftKeySelector.asInstanceOf[ResultTypeQueryable[_]].getProducedType)
+    ret
+  }
+
+  private def validateKeyTypes(): Unit = {
+    // at least one equality expression
+    val leftFields = left.getRowType.getFieldList
+    val rightFields = right.getRowType.getFieldList
+
+    getJoinInfo.pairs().toList.foreach(pair => {
+      val leftKeyType = leftFields.get(pair.source).getType.getSqlTypeName
+      val rightKeyType = rightFields.get(pair.target).getType.getSqlTypeName
+      // check if keys are compatible
+      if (leftKeyType != rightKeyType) {
+        throw new TableException(
+          "Equality join predicate on incompatible types.\n" +
+            s"\tLeft: $left,\n" +
+            s"\tRight: $right,\n" +
+            s"\tCondition: (${RelExplainUtil.expressionToString(
+              getCondition, inputRowType, getExpressionString)})"
+        )
+      }
+    })
+  }
+
+  private def getJoinOperatorName: String = {
+    val where = RelExplainUtil.expressionToString(getCondition, inputRowType, 
getExpressionString)
+    val select = getRowType.getFieldNames.mkString(", ")
+    s"TemporalTableJoin(where: ($where), select: ($select)"
+  }
+}
+
+
+/**
+  * @param rightTimeAttributeInputReference is defined only for event time 
joins.
+  */
+class StreamExecTemporalJoinToCoProcessTranslator private (
+  textualRepresentation: String,
+  config: TableConfig,
+  returnType: RowType,
+  leftInputType: RowType,
+  rightInputType: RowType,
+  joinInfo: JoinInfo,
+  rexBuilder: RexBuilder,
+  leftTimeAttributeInputReference: Int,
+  rightTimeAttributeInputReference: Option[Int],
+  remainingNonEquiJoinPredicates: RexNode) {
+
+  val nonEquiJoinPredicates: Option[RexNode] = 
Some(remainingNonEquiJoinPredicates)
+
+  def getLeftKeySelector: BaseRowKeySelector = {
+    KeySelectorUtil.getBaseRowSelector(
+      joinInfo.leftKeys.toIntArray,
+      BaseRowTypeInfo.of(leftInputType)
+    )
+  }
+
+  def getRightKeySelector: BaseRowKeySelector = {
+    KeySelectorUtil.getBaseRowSelector(
+      joinInfo.rightKeys.toIntArray,
+      BaseRowTypeInfo.of(rightInputType)
+    )
+  }
+
+  def getJoinOperator(
+    joinType: JoinRelType,
+    returnFieldNames: Seq[String]): TwoInputStreamOperator[BaseRow, BaseRow, 
BaseRow] = {
+
+    // input must not be nullable, because the runtime join function will make 
sure
+    // the code-generated function won't process null inputs
+    val ctx = CodeGeneratorContext(config)
+    val exprGenerator = new ExprCodeGenerator(ctx, nullableInput = false)
+      .bindInput(leftInputType)
+      .bindSecondInput(rightInputType)
+
+    val body = if (nonEquiJoinPredicates.isEmpty) {
+      // only equality condition
+      "return true;"
+    } else {
+      val condition = 
exprGenerator.generateExpression(nonEquiJoinPredicates.get)
+      s"""
+         |${condition.code}
+         |return ${condition.resultTerm};
+         |""".stripMargin
+    }
+
+    val generatedJoinCondition = FunctionCodeGenerator.generateJoinCondition(
+      ctx,
+      "ConditionFunction",
+      body)
+
+    createJoinOperator(config, joinType, generatedJoinCondition)
+  }
+
+  protected def createJoinOperator(
+    tableConfig: TableConfig,
+    joinType: JoinRelType,
+    generatedJoinCondition: GeneratedJoinCondition)
+  : TwoInputStreamOperator[BaseRow, BaseRow, BaseRow] = {
+
+    val minRetentionTime = tableConfig.getMinIdleStateRetentionTime
 
 Review comment:
   Put these two variables to `new TemporalProcessTimeJoinOperator` block?

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