[
https://issues.apache.org/jira/browse/FLINK-6232?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16085364#comment-16085364
]
ASF GitHub Bot commented on FLINK-6232:
---------------------------------------
Github user wuchong commented on a diff in the pull request:
https://github.com/apache/flink/pull/4266#discussion_r126671328
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
---
@@ -0,0 +1,187 @@
+/*
+ * 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.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin,
WindowJoinUtil}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.updateutils.UpdateCheckUtils
+
+/**
+ * Flink RelNode which matches along with JoinOperator and its related
operations.
+ */
+class DataStreamWindowJoin(
+ cluster: RelOptCluster,
+ traitSet: RelTraitSet,
+ leftNode: RelNode,
+ rightNode: RelNode,
+ joinCondition: RexNode,
+ joinType: JoinRelType,
+ leftSchema: RowSchema,
+ rightSchema: RowSchema,
+ schema: RowSchema,
+ isRowTime: Boolean,
+ leftLowerBound: Long,
+ leftUpperBound: Long,
+ remainCondition: Option[RexNode],
+ 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 DataStreamWindowJoin(
+ cluster,
+ traitSet,
+ inputs.get(0),
+ inputs.get(1),
+ joinCondition,
+ joinType,
+ leftSchema,
+ rightSchema,
+ schema,
+ isRowTime,
+ leftLowerBound,
+ leftUpperBound,
+ remainCondition,
+ ruleDescription)
+ }
+
+ override def toString: String = {
+ joinToString(
+ schema.logicalType,
+ joinCondition,
+ joinType,
+ getExpressionString)
+ }
+
+ override def explainTerms(pw: RelWriter): RelWriter = {
+ joinExplainTerms(
+ super.explainTerms(pw),
+ schema.logicalType,
+ joinCondition,
+ joinType,
+ getExpressionString)
+ }
+
+ override def translateToPlan(
+ tableEnv: StreamTableEnvironment,
+ queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+ val config = tableEnv.getConfig
+
+ val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
+ val isRightAppendOnly = UpdateCheckUtils.isAppendOnly(right)
+ if (!isLeftAppendOnly || !isRightAppendOnly) {
+ throw new TableException(
+ "Windowed stream join does not support updates.")
+ }
+
+ val leftDataStream =
left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+ val rightDataStream =
right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+
+ // get the equality keys and other condition
+ val joinInfo = JoinInfo.of(leftNode, rightNode, joinCondition)
+ val leftKeys = joinInfo.leftKeys.toIntArray
+ val rightKeys = joinInfo.rightKeys.toIntArray
+
+ // generate join function
+ val joinFunction =
+ WindowJoinUtil.generateJoinFunction(
+ config,
+ joinType,
+ leftSchema.physicalTypeInfo,
+ rightSchema.physicalTypeInfo,
+ schema,
+ remainCondition,
+ ruleDescription)
+
+ joinType match {
+ case JoinRelType.INNER =>
+ isRowTime match {
--- End diff --
I think using a `if (isRowTime) else ` here is more simple.
> 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)