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

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_r117006282
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
    @@ -0,0 +1,488 @@
    +/*
    + * 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
    +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.{CodeGenerator, ExpressionReducer}
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.runtime.FilterRunner
    +import org.apache.flink.types.Row
    +
    +import scala.collection.JavaConversions._
    +import scala.collection.mutable.ArrayBuffer
    +
    +
    +object JoinUtil {
    +
    +  /**
    +    * Analyze join condition to get equi-conditon and other condition
    +    * @param  joinNode   logicaljoin node
    +    * @param  expression the function to generate condition string
    +    */
    +  private[flink] def analyzeJoinCondition(
    +    joinNode: FlinkLogicalJoin,
    +    expression: (RexNode, List[String], Option[List[RexNode]]) => String) 
= {
    +
    +    val joinInfo = joinNode.analyzeCondition()
    +    val keyPairs = joinInfo.pairs.toList
    +    val otherCondition =
    +      if(joinInfo.isEqui) null
    +      else joinInfo.getRemaining(joinNode.getCluster.getRexBuilder)
    +
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (!keyPairs.isEmpty) {
    +      val leftFields = joinNode.getLeft.getRowType.getFieldList
    +      val rightFields = joinNode.getRight.getRowType.getFieldList
    +
    +      keyPairs.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) {
    +          // add key pair
    +          leftKeys.append(pair.source)
    +          rightKeys.append(pair.target)
    +        } else {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${joinNode.getLeft.toString},\n" +
    +              s"\tRight: ${joinNode.getRight.toString},\n" +
    +              s"\tCondition: (${expression(joinNode.getCondition,
    +                joinNode.getRowType.getFieldNames.toList, None)})"
    +          )
    +        }
    +      })
    +    }
    +    (leftKeys.toArray, rightKeys.toArray, otherCondition)
    +  }
    +
    +  /**
    +    * 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(
    --- End diff --
    
    I think the logic of this function is correct. However, I find it a bit 
hard to follow because it starts with many conditions.
    What do you think about the following approach:
    
    1. convert condition to CNF
    2. go over all conjunctive terms and split them into those referencing a 
time field and those that don't reference a time field (those latter terms are 
the remaining conditions)
    3. Check that there are only two such terms, throw an exception otherwise
    4. Check for each conjunctive term that is references time fields of both 
inputs exactly once. If there is one that does not, throw an exception.
    5. Extract the offset from each of the two terms (this is the complex step)
    
    This way, we keep the logic simple until the end and don't need to 
distinguish between greater / smaller, left input, right input, etc. from the 
start.
    
    What do you think?



> 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