[
https://issues.apache.org/jira/browse/FLINK-6232?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16085363#comment-16085363
]
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_r126680468
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
---
@@ -0,0 +1,349 @@
+/*
+ * 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 org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.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.schema.{RowSchema,
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+ * An util class to help analyze and build join code .
+ */
+object WindowJoinUtil {
+
+ /**
+ * Analyze time-condtion to get time boundary for each stream and get
the time type
+ * and return remain condition.
+ *
+ * @param condition join condition
+ * @param leftLogicalFieldCnt left stream logical field num
+ * @param inputSchema join rowtype schema
+ * @param rexBuilder util to build rexNode
+ * @param config table environment config
+ * @return isRowTime, left lower boundary, right lower boundary, remain
condition
+ */
+ private[flink] def analyzeTimeBoundary(
+ condition: RexNode,
+ leftLogicalFieldCnt: Int,
+ inputSchema: RowSchema,
+ rexBuilder: RexBuilder,
+ config: TableConfig): (Boolean, Long, Long, Option[RexNode]) = {
+
+ // Converts the condition to conjunctive normal form (CNF)
+ val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+ // split the condition into time indicator condition and other
condition
+ val (timeTerms, remainTerms) = cnfCondition match {
+ case c: RexCall if cnfCondition.getKind == SqlKind.AND =>
+ c.getOperands.asScala
+ .map(analyzeCondtionTermType(_, leftLogicalFieldCnt,
inputSchema.logicalType))
+ .reduceLeft((l, r) => {
+ (l._1 ++ r._1, l._2 ++ r._2)
+ })
+ case _ =>
+ throw new TableException("A time-based stream join requires
exactly " +
+ "two join predicates that bound the time in both directions.")
+ }
+
+ if (timeTerms.size != 2) {
+ throw new TableException("A time-based stream join requires exactly
" +
+ "two join predicates that bound the time in both directions.")
+ }
+
+ // extract time offset from the time indicator conditon
--- End diff --
minor typo: conditon -> condition
> 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)