You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/07/03 02:48:00 UTC

[jira] [Commented] (FLINK-6232) Support proctime inner equi-join between two streams in the SQL API

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

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

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

    https://github.com/apache/flink/pull/3715#discussion_r125202807
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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 org.apache.calcite.plan.RelOptUtil
    +import org.apache.calcite.rel.`type`.RelDataType
    +import org.apache.calcite.rel.core.JoinRelType
    +import org.apache.calcite.rex._
    +import org.apache.calcite.sql.SqlKind
    +import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.table.api.{TableConfig, TableException}
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
    +import org.apache.flink.table.functions.TimeMaterializationSqlFunction
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.JavaConversions._
    +
    +/**
    +  * An util class to help analyze and build join code .
    +  */
    +object JoinUtil {
    +
    +  /**
    +    * check if the join case is stream join stream
    +    *
    +    * @param  condition   other condtion include time-condition
    +    * @param  inputType   left and right connect stream type
    +    */
    +  private[flink] def isStreamStreamJoin(
    +      condition: RexNode,
    +      inputType: RelDataType) = {
    +
    +    def isExistTumble(expr: RexNode): Boolean = {
    +      expr match {
    +        case c: RexCall =>
    +          c.getOperator match {
    +            case _: SqlFloorFunction =>
    +              c.getOperands.map(analyzeSingleConditionTerm(_, 0, inputType)).exists(_.size > 0)
    +            case SqlStdOperatorTable.TUMBLE =>
    +              c.getOperands.map(analyzeSingleConditionTerm(_, 0, inputType)).exists(_.size > 0)
    +            case _ =>
    +              c.getOperands.map(isExistTumble(_)).exists(_ == true)
    +          }
    +        case _ => false
    +      }
    +    }
    +
    +    val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, inputType).size > 0
    +    val isExistTumbleExpr = isExistTumble(condition)
    +
    +    !isExistTumbleExpr && isExistTimeIndicator
    +  }
    +
    +  /**
    +    * Analyze time-condtion to get time boundary for each stream and get the time type
    +    * and return remain condition.
    +    *
    +    * @param  condition   other condtion include time-condition
    +    * @param  leftLogicalFieldCnt left stream logical field num
    +    * @param  leftPhysicalFieldCnt left stream physical field num
    +    * @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, Option[RexNode]) = {
    +
    +    // Converts the condition to conjunctive normal form (CNF)
    +    val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
    +
    +    // split the condition into time indicator condition and other condition
    +    val (timeTerms, remainTerms) =
    +      splitJoinCondition(
    +        cnfCondition,
    +        leftLogicalFieldCnt,
    +        inputType
    +      )
    +
    +    if (timeTerms.size != 2) {
    +      throw new TableException("There only can and must have 2 time conditions.")
    +    }
    +
    +    // extract time offset from the time indicator conditon
    +    val streamTimeOffsets =
    +      timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, rexBuilder, config))
    +
    +    val (leftTableOffset, rightTableOffset) =
    +      streamTimeOffsets match {
    +        case Seq((x, true), (y, false)) => (x, y)
    +        case Seq((x, false), (y, true)) => (y, x)
    +        case _ =>
    +          throw new TableException("Both input need time boundary.")
    +      }
    +
    +    // compose the remain condition list into one condition
    +    val remainCondition =
    +      remainTerms match {
    +        case Seq() => None
    +        case _ =>
    +          // turn the logical field index to physical field index
    +          def transInputRef(expr: RexNode): RexNode = {
    +            expr match {
    +              case c: RexCall =>
    +                val newOps = c.operands.map(transInputRef(_))
    +                rexBuilder.makeCall(c.getType, c.getOperator, newOps)
    +              case i: RexInputRef if i.getIndex >= leftLogicalFieldCnt =>
    +                rexBuilder.makeInputRef(
    +                  i.getType,
    +                  i.getIndex - leftLogicalFieldCnt + leftPhysicalFieldCnt)
    +              case _ => expr
    +            }
    +          }
    +
    +          Some(remainTerms.map(transInputRef(_)).reduceLeft( (l, r) => {
    --- End diff --
    
    The index of the RexInputRef is the index combine two stream. For example, stream A(id, attr1, attr2, attr3) join stream B(id, attr1), then the condition a.id > b.id, the b.id's index will be 4 but not 0, so we need to traverse and convert it to 0, thus in the code-gen join function, we can use in2.getField(0) to get the value.


> 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.4.14#64029)