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

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_r122843078
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRowStreamJoin.scala
 ---
    @@ -0,0 +1,186 @@
    +/*
    + * 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.datastream
    +
    +import org.apache.calcite.plan._
    +import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
    +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
    +import org.apache.calcite.rex.RexNode
    +import org.apache.flink.api.java.functions.NullByteKeySelector
    +import org.apache.flink.streaming.api.datastream.DataStream
    +import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.CommonJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
    +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
    +
    +/**
    +  * Flink RelNode which matches along with JoinOperator and its related 
operations.
    +  */
    +class DataStreamRowStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +  with CommonJoin
    +  with DataStreamRel {
    +
    +  override def deriveRowType() = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
    +    new DataStreamRowStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  override def toString: String = {
    +
    +    s"${joinTypeToString(joinType)}" +
    +      s"(condition: (${joinConditionToString(schema.logicalType,
    +        joinCondition, getExpressionString)}), " +
    +      s"select: (${joinSelectionToString(schema.logicalType)}))"
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    super.explainTerms(pw)
    +      .item("condition", joinConditionToString(schema.logicalType,
    +        joinCondition, getExpressionString))
    +      .item("select", joinSelectionToString(schema.logicalType))
    +      .item("joinType", joinTypeToString(joinType))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +
    +    // get the equality keys and other condition
    +    val joinInfo = JoinInfo.of(leftNode, rightNode, joinCondition)
    +    val leftKeys = joinInfo.leftKeys.toIntArray
    +    val rightKeys = joinInfo.rightKeys.toIntArray
    +    val otherCondition = joinInfo.getRemaining(cluster.getRexBuilder)
    +
    +    // analyze time boundary and time predicate type(proctime/rowtime)
    +    val (timeType, leftStreamWindowSize, rightStreamWindowSize, 
remainCondition) =
    +      JoinUtil.analyzeTimeBoundary(
    --- End diff --
    
    I think we should move the analysis to the rule. Otherwise, we might end up 
with a plan that cannot be translated. It is the rule's responsibility to 
ensure that the translated plan can be executed.
    
    The rule can then pass the analyzed time predicate parameters (time type, 
bounds) to the `DataStreamRowStreamJoin`


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

Reply via email to