[ https://issues.apache.org/jira/browse/FLINK-6232?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16005627#comment-16005627 ]
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_r115874222 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala --- @@ -0,0 +1,180 @@ +/* + * 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.JoinRelType +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter} +import org.apache.flink.api.common.functions.RichFilterFunction +import org.apache.flink.api.java.functions.NullByteKeySelector +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.table.api.{StreamTableEnvironment, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.plan.nodes.CommonJoin +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin +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} +import org.apache.flink.types.Row + +/** + * Flink RelNode which matches along with JoinOperator and its related operations. + */ +class DataStreamJoin( + cluster: RelOptCluster, + traitSet: RelTraitSet, + leftNode: RelNode, + rightNode: RelNode, + joinNode: FlinkLogicalJoin, + leftSchema: 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 DataStreamJoin( + cluster, + traitSet, + inputs.get(0), + inputs.get(1), + joinNode, + leftSchema, + schema, + ruleDescription) + } + + override def toString: String = { + + s"${joinTypeToString(joinNode.getJoinType)}" + + s"(condition: (${joinConditionToString(schema.logicalType, + joinNode.getCondition, getExpressionString)}), " + + s"select: (${joinSelectionToString(schema.logicalType)}))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + super.explainTerms(pw) + .item("condition", joinConditionToString(schema.logicalType, + joinNode.getCondition, getExpressionString)) + .item("select", joinSelectionToString(schema.logicalType)) + .item("joinType", joinTypeToString(joinNode.getJoinType)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + + val config = tableEnv.getConfig + + // get the equality keys and other condition + val (leftKeys, rightKeys, otherCondition) = + JoinUtil.analyzeJoinCondition(joinNode, getExpressionString) + + if (left.isInstanceOf[StreamTableSourceScan] + || right.isInstanceOf[StreamTableSourceScan]) { + throw new TableException( + "Join between stream and table is not supported yet.") + } + // analyze time boundary and time predicate type(proctime/rowtime) + val (timeType, leftStreamWindowSize, rightStreamWindowSize, conditionWithoutTime) = + JoinUtil.analyzeTimeBoundary( + otherCondition, + leftSchema.logicalType.getFieldCount, + leftSchema.physicalType.getFieldCount, + schema.logicalType, + joinNode.getCluster.getRexBuilder, + config) + + val leftDataStream = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val rightDataStream = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + + // generate other condition filter function + val filterFunction = + JoinUtil.generateFilterFunction( + config, + joinNode.getJoinType, + CRowTypeInfo(schema.physicalTypeInfo).rowType, + conditionWithoutTime, + ruleDescription) + + joinNode.getJoinType match { + case JoinRelType.INNER => + timeType match { + case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType) => + // Proctime JoinCoProcessFunction + createProcTimeInnerJoinFunction( + leftStreamWindowSize, + rightStreamWindowSize, + leftDataStream, + rightDataStream, + filterFunction, + leftKeys, + rightKeys + ) + case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) => + // RowTime JoinCoProcessFunction + throw new TableException( + "RowTime inner join between stream and stream is not supported yet.") + } + case JoinRelType.FULL => + throw new TableException( + "Full join between stream and stream is not supported yet.") + case JoinRelType.LEFT => + throw new TableException( + "Left join between stream and stream is not supported yet.") + case JoinRelType.RIGHT => + throw new TableException( + "Right join between stream and stream is not supported yet.") + } + } + + def createProcTimeInnerJoinFunction( + leftStreamWindowSize: Long, + rightStreamWindowSize: Long, + leftDataStream: DataStream[CRow], + rightDataStream: DataStream[CRow], + filterFunction: RichFilterFunction[Row], + leftKeys: Array[Int], + rightKeys: Array[Int]): DataStream[CRow] = { + + val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + + val procInnerJoinFunc = new ProcTimeInnerJoin( + leftStreamWindowSize, + rightStreamWindowSize, + leftDataStream.getType, + rightDataStream.getType, + filterFunction) + + if (!leftKeys.isEmpty) { + leftDataStream.connect(rightDataStream) + .keyBy(leftKeys, rightKeys) + .process(procInnerJoinFunc) + .returns(returnTypeInfo) + } else { + leftDataStream.connect(rightDataStream) --- End diff -- If we support this case, we should also have a test for it. > 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)