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

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

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3715#discussion_r116794639
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
    @@ -0,0 +1,468 @@
    +/*
    + * 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.runtime.join
    +
    +import java.math.{BigDecimal => JBigDecimal}
    +import java.util
    +import java.util.EnumSet
    +
    +import org.apache.calcite.avatica.util.TimeUnit
    +import org.apache.calcite.rel.`type`.RelDataType
    +import org.apache.calcite.rel.core.JoinRelType
    +import org.apache.calcite.rex._
    +import org.apache.calcite.sql.fun.SqlStdOperatorTable
    +import org.apache.calcite.sql.parser.SqlParserPos
    +import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
    +import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
    +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
    +import org.apache.flink.table.api.{TableConfig, TableException}
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.types.Row
    +
    +import scala.collection.JavaConversions._
    +import scala.collection.mutable.ArrayBuffer
    +
    +
    +object JoinUtil {
    +
    +  /**
    +    * Analyze time-condtion to get time boundary for each stream and get 
the time type
    +    * and return condition without time-condition.
    +    *
    +    * @param  condition   other condtion include time-condition
    +    * @param  leftFieldCount left stream fields count
    +    * @param  inputType   left and right connect stream type
    +    * @param  rexBuilder   util to build rexNode
    +    * @param  config      table environment config
    +    */
    +  private[flink] def analyzeTimeBoundary(
    +      condition: RexNode,
    +      leftLogicalFieldCnt: Int,
    +      leftPhysicalFieldCnt: Int,
    +      inputType: RelDataType,
    +      rexBuilder: RexBuilder,
    +      config: TableConfig): (RelDataType, Long, Long, RexNode) = {
    +    // analyze the time-conditon to get greate and less condition,
    +    // make sure left stream field in the left of the condition
    +    // e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
    +    val greateConditions = new util.ArrayList[TimeSingleCondition]()
    +    val lessConditions = new util.ArrayList[TimeSingleCondition]()
    +    analyzeTimeCondition(condition, greateConditions,
    +      lessConditions, leftLogicalFieldCnt, inputType)
    +    if (greateConditions.size != lessConditions.size
    +        || greateConditions.size > 1
    +        || greateConditions.size == 0) {
    +      throw TableException(
    +        "Equality join time conditon should have proctime or rowtime 
indicator."
    +      )
    +    }
    +
    +    val greatCond = greateConditions.get(0)
    +    val lessCond = lessConditions.get(0)
    +    if (greatCond.timeType != lessCond.timeType) {
    +      throw TableException(
    +        "Equality join time conditon should all use proctime or all use 
rowtime."
    +      )
    +    }
    +
    +    var leftStreamWindowSize: Long = 0
    +    var rightStreamWindowSize: Long = 0
    +
    +    // only a.proctime > b.proctime - interval '1' hour need to store a 
stream
    +    val timeLiteral: RexLiteral =
    +        reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
    +    leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
    +    // only need to store past records
    +    if (leftStreamWindowSize < 0) {
    +      leftStreamWindowSize = -leftStreamWindowSize
    +      if (!greatCond.isEqual) {
    +        leftStreamWindowSize -= 1
    +      }
    +    } else {
    +      leftStreamWindowSize = 0
    +    }
    +
    +    // only a.proctime < b.proctime + interval '1' hour need to store b 
stream
    +    val timeLiteral2: RexLiteral =
    +        reduceTimeExpression(lessCond.leftExpr, lessCond.rightExpr, 
rexBuilder, config)
    +    rightStreamWindowSize = timeLiteral2.getValue2.asInstanceOf[Long]
    +    // only need to store past records
    +    if (rightStreamWindowSize < 0) {
    +      rightStreamWindowSize = -rightStreamWindowSize
    +      if (!lessCond.isEqual) {
    +        rightStreamWindowSize -= 1
    +      }
    +    } else {
    +      rightStreamWindowSize = 0
    +    }
    +
    +    // get condition without time-condition
    +    // e.g a.price > b.price and a.proctime between b.proctime and 
b.proctime + interval '1' hour
    +    // will return a.price > b.price and true and true
    +    var conditionWithoutTime = removeTimeCondition(
    +      condition,
    +      greatCond.originCall,
    +      lessCond.originCall,
    +      rexBuilder,
    +      leftLogicalFieldCnt,
    +      leftPhysicalFieldCnt)
    +
    +    // reduce the expression
    +    // true and ture => true, otherwise keep the origin expression
    +    try {
    +      val exprReducer = new ExpressionReducer(config)
    +      val originList = new util.ArrayList[RexNode]()
    +      originList.add(conditionWithoutTime)
    +      val reduceList = new util.ArrayList[RexNode]()
    +      exprReducer.reduce(rexBuilder, originList, reduceList)
    +      conditionWithoutTime = reduceList.get(0)
    +    } catch {
    +      case _ : CodeGenException => // ignore
    +    }
    +
    +    (greatCond.timeType, leftStreamWindowSize, rightStreamWindowSize, 
conditionWithoutTime)
    +  }
    +
    +  /**
    +    * Generate other non-equi condition function
    +    * @param  config   table env config
    +    * @param  joinType  join type to determain whether input can be null
    +    * @param  leftType  left stream type
    +    * @param  rightType  right stream type
    +    * @param  returnType   return type
    +    * @param  otherCondition   non-equi condition
    +    * @param  ruleDescription  rule description
    +    */
    +  private[flink] def generateJoinFunction(
    +    config: TableConfig,
    +    joinType: JoinRelType,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    returnType: RowSchema,
    +    otherCondition: RexNode,
    +    ruleDescription: String) = {
    +
    +    // whether input can be null
    +    val nullCheck = joinType match {
    +      case JoinRelType.INNER => false
    +      case JoinRelType.LEFT  => true
    +      case JoinRelType.RIGHT => true
    +      case JoinRelType.FULL  => true
    +    }
    +
    +    // generate other non-equi function code
    +    val generator = new CodeGenerator(
    +      config,
    +      nullCheck,
    +      leftType,
    +      Some(rightType))
    +
    +    val conversion = generator.generateConverterResultExpression(
    +      returnType.physicalTypeInfo,
    +      returnType.physicalType.getFieldNames)
    +
    +    // if other condition is literal(true), then output the result directly
    +    val body = if (otherCondition.isAlwaysTrue) {
    +      s"""
    +         |${conversion.code}
    +         |${generator.collectorTerm}.collect(${conversion.resultTerm});
    +         |""".stripMargin
    +    }
    +    else {
    +      val condition = generator.generateExpression(otherCondition)
    +      s"""
    +         |${condition.code}
    +         |if (${condition.resultTerm}) {
    +         |  ${conversion.code}
    +         |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
    +         |}
    +         |""".stripMargin
    +    }
    +
    +    val genFunction = generator.generateFunction(
    +      ruleDescription,
    +      classOf[FlatJoinFunction[Row, Row, Row]],
    +      body,
    +      returnType.physicalTypeInfo)
    +
    +    genFunction
    +  }
    +
    +  private case class TimeSingleCondition(
    +      timeType: RelDataType,
    +      leftExpr: RexNode,
    +      rightExpr: RexNode,
    +      isEqual: Boolean,
    +      originCall: RexNode)
    +
    +  val COMPARISON: util.Set[SqlKind] = EnumSet.of(
    +    SqlKind.LESS_THAN,
    +    SqlKind.GREATER_THAN,
    +    SqlKind.GREATER_THAN_OR_EQUAL,
    +    SqlKind.LESS_THAN_OR_EQUAL)
    +
    +  val EQUI_COMPARISON: util.Set[SqlKind] = EnumSet.of(
    +    SqlKind.GREATER_THAN_OR_EQUAL,
    +    SqlKind.LESS_THAN_OR_EQUAL)
    +
    +  val LESS_COMPARISON: util.Set[SqlKind] = EnumSet.of(
    +    SqlKind.LESS_THAN,
    +    SqlKind.LESS_THAN_OR_EQUAL)
    +
    +  val GREAT_COMPARISON: util.Set[SqlKind] = EnumSet.of(
    +    SqlKind.GREATER_THAN,
    +    SqlKind.GREATER_THAN_OR_EQUAL)
    +
    +  /**
    +    * Analyze time-conditon to divide all time-condition into great and 
less condition
    +    */
    +  private def analyzeTimeCondition(
    +    condition: RexNode,
    --- End diff --
    
    please indent the arguments by two more spaces and add a new line to make 
the signature better distinguishable from the body:+
    
    ```
    private def analyzeTimeCondition(
        condition: RexNode,
        greatCondition: util.List[TimeSingleCondition],
        lessCondition: util.List[TimeSingleCondition],
        leftFieldCount: Int,
        inputType: RelDataType): Unit = {
    
      if (condition.isInstanceOf[RexCall]) {
      ...
    ```


> Support proctime inner equi-join between two streams in the SQL API
> -------------------------------------------------------------------
>
>                 Key: FLINK-6232
>                 URL: https://issues.apache.org/jira/browse/FLINK-6232
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: hongyuhong
>            Assignee: hongyuhong
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to