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

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_r116791596
  
    --- 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,
    --- End diff --
    
    Please wrap the arguments of a function as follows (if it does not fit in a 
single line):
    ```
    analyzeTimeCondition(
      condition, 
      greateConditions,
      lessConditions, 
      leftLogicalFieldCnt, 
      inputType)
    ```


> 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