[
https://issues.apache.org/jira/browse/FLINK-6233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16155158#comment-16155158
]
ASF GitHub Bot commented on FLINK-6233:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/4625#discussion_r137229663
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo,
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import
org.apache.flink.table.runtime.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+ * A CoProcessFunction to execute time-bounded stream inner-join.
+ *
+ * Sample criteria:
+ *
+ * L.time between R.time + X and R.time + Y
+ * or AND R.time between L.time - Y and L.time - X
+ *
+ * @param leftLowerBound X
+ * @param leftUpperBound Y
+ * @param allowedLateness the lateness allowed for the two streams
+ * @param leftType the input type of left stream
+ * @param rightType the input type of right stream
+ * @param genJoinFuncName the function code of other non-equi conditions
+ * @param genJoinFuncCode the function name of other non-equi conditions
+ * @param timeIndicator indicate whether joining on proctime or rowtime
+ *
+ */
+class TimeBoundedStreamInnerJoin(
+ private val leftLowerBound: Long,
+ private val leftUpperBound: Long,
+ private val allowedLateness: Long,
+ private val leftType: TypeInformation[Row],
+ private val rightType: TypeInformation[Row],
+ private val genJoinFuncName: String,
+ private val genJoinFuncCode: String,
+ private val leftTimeIdx: Int,
+ private val rightTimeIdx: Int,
+ private val timeIndicator: JoinTimeIndicator)
+ extends CoProcessFunction[CRow, CRow, CRow]
+ with Compiler[FlatJoinFunction[Row, Row, Row]]
+ with Logging {
+
+ private var cRowWrapper: CRowWrappingCollector = _
+
+ // the join function for other conditions
+ private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+ // cache to store the left stream records
+ private var leftCache: MapState[Long, JList[Row]] = _
+ // cache to store right stream records
+ private var rightCache: MapState[Long, JList[Row]] = _
+
+ // state to record the timer on the left stream. 0 means no timer set
+ private var leftTimerState: ValueState[Long] = _
+ // state to record the timer on the right stream. 0 means no timer set
+ private var rightTimerState: ValueState[Long] = _
+
+ private val leftRelativeSize: Long = -leftLowerBound
+ private val rightRelativeSize: Long = leftUpperBound
+
+ private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+ private var leftOperatorTime: Long = 0L
+ private var rightOperatorTime: Long = 0L
+
+ private var backPressureSuggestion: Long = 0L
+
+ if (relativeWindowSize <= 0) {
+ LOG.warn("The relative window size is non-positive, please check the
join conditions.")
+ }
+
+ if (allowedLateness < 0) {
+ throw new IllegalArgumentException("The allowed lateness must be
non-negative.")
+ }
+
+
+ /**
+ * For holding back watermarks.
+ *
+ * @return the maximum delay for the outputs
+ */
+ def getMaxOutputDelay = Math.max(leftRelativeSize, rightRelativeSize) +
allowedLateness;
+
+ /**
+ * For dynamic query optimization.
+ *
+ * @return the suggested offset time for back-pressure
+ */
+ def getBackPressureSuggestion = backPressureSuggestion
+
+ override def open(config: Configuration) {
+ val clazz = compile(
+ getRuntimeContext.getUserCodeClassLoader,
+ genJoinFuncName,
+ genJoinFuncCode)
+ joinFunction = clazz.newInstance()
+
+ cRowWrapper = new CRowWrappingCollector()
+ cRowWrapper.setChange(true)
+
+ // Initialize the data caches.
+ val leftListTypeInfo: TypeInformation[JList[Row]] = new
ListTypeInfo[Row](leftType)
+ val leftStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+ new MapStateDescriptor[Long, JList[Row]](timeIndicator +
"InnerJoinLeftCache",
+ BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]],
leftListTypeInfo)
+ leftCache = getRuntimeContext.getMapState(leftStateDescriptor)
+
+ val rightListTypeInfo: TypeInformation[JList[Row]] = new
ListTypeInfo[Row](rightType)
+ val rightStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+ new MapStateDescriptor[Long, JList[Row]](timeIndicator +
"InnerJoinRightCache",
+ BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]],
rightListTypeInfo)
+ rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
+
+ // Initialize the timer states.
+ val leftTimerStateDesc: ValueStateDescriptor[Long] =
+ new ValueStateDescriptor[Long](timeIndicator +
"InnerJoinLeftTimerState",
+ classOf[Long])
+ leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
+
+ val rightTimerStateDesc: ValueStateDescriptor[Long] =
+ new ValueStateDescriptor[Long](timeIndicator +
"InnerJoinRightTimerState",
+ classOf[Long])
+ rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
+ }
+
+ /**
+ * Process records from the left stream.
+ *
+ * @param cRowValue the input record
+ * @param ctx the context to register timer or get current time
+ * @param out the collector for outputting results
+ *
+ */
+ override def processElement1(
+ cRowValue: CRow,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow]): Unit = {
+ val timeForRecord: Long = getTimeForRecord(ctx, cRowValue, true)
+ getCurrentOperatorTime(ctx)
+ processElement(
+ cRowValue,
+ timeForRecord,
+ ctx,
+ out,
+ leftOperatorTime,
+ rightOperatorTime,
+ rightTimerState,
+ leftCache,
+ rightCache,
+ true
+ )
+ }
+
+ /**
+ * Process records from the right stream.
+ *
+ * @param cRowValue the input record
+ * @param ctx the context to get current time
+ * @param out the collector for outputting results
+ *
+ */
+ override def processElement2(
+ cRowValue: CRow,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow]): Unit = {
+ val timeForRecord: Long = getTimeForRecord(ctx, cRowValue, false)
+ getCurrentOperatorTime(ctx)
+ processElement(
+ cRowValue,
+ timeForRecord,
+ ctx,
+ out,
+ rightOperatorTime,
+ leftOperatorTime,
+ leftTimerState,
+ rightCache,
+ leftCache,
+ false
+ )
+ }
+
+ /**
+ * Put a record from the input stream into the cache and iterate the
opposite cache to
+ * output records meeting the join conditions. If there is no timer set
for the OPPOSITE
+ * STREAM, register one.
+ */
+ private def processElement(
+ cRowValue: CRow,
+ timeForRecord: Long,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow],
+ myWatermark: Long,
+ oppositeWatermark: Long,
+ oppositeTimeState: ValueState[Long],
+ recordListCache: MapState[Long, JList[Row]],
+ oppositeCache: MapState[Long, JList[Row]],
+ leftRecord: Boolean): Unit = {
+ if (relativeWindowSize > 0) {
+ //TODO Shall we consider adding a method for initialization with the
context and collector?
+ cRowWrapper.out = out
+
+ val record = cRowValue.row
+
+ //TODO Only if the time of the record is greater than the watermark,
can we continue.
+ if (timeForRecord >= myWatermark - allowedLateness) {
+ val oppositeLowerBound: Long =
+ if (leftRecord) timeForRecord - rightRelativeSize else
timeForRecord - leftRelativeSize
+
+ val oppositeUpperBound: Long =
+ if (leftRecord) timeForRecord + leftRelativeSize else
timeForRecord + rightRelativeSize
+
+ // Put the record into the cache for later use.
+ val recordList = if (recordListCache.contains(timeForRecord)) {
+ recordListCache.get(timeForRecord)
+ } else {
+ new util.ArrayList[Row]()
+ }
+ recordList.add(record)
+ recordListCache.put(timeForRecord, recordList)
+
+ // Register a timer on THE OTHER STREAM to remove records from the
cache once they are
+ // expired.
+ if (oppositeTimeState.value == 0) {
+ registerCleanUpTimer(
+ ctx, timeForRecord, oppositeWatermark, oppositeTimeState,
leftRecord, true)
+ }
+
+ // Join the record with records from the opposite stream.
+ val oppositeIterator = oppositeCache.iterator()
+ var oppositeEntry: Entry[Long, util.List[Row]] = null
+ var oppositeTime: Long = 0L;
+ while (oppositeIterator.hasNext) {
+ oppositeEntry = oppositeIterator.next
+ oppositeTime = oppositeEntry.getKey
+ if (oppositeTime < oppositeLowerBound - allowedLateness) {
+ //TODO Considering the data out-of-order, we should not remove
records here.
--- End diff --
We can add a cleanup condition which is based on the watermark and not on
the current records timestamps. If we make the cleanup timer a safety net with
larger intervals, we can also clean up during joining. Should be more efficient
since we go over the full MapState anyway.
> Support rowtime inner equi-join between two streams in the SQL API
> ------------------------------------------------------------------
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: hongyuhong
> Assignee: Xingcan Cui
>
> 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.rowtime , o.productId, o.orderId, s.rowtime AS shipTime
> FROM Orders AS o
> JOIN Shipments AS s
> ON o.orderId = s.orderId
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + 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.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL
> '1' HOUR}} only can use rowtime that is a system attribute, the time
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support
> unbounded like {{o.rowtime < s.rowtime}} , and should include both two
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () +
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this
> would mean in insert a row into a sorted order shift all other computations.
> This would be too expensive to maintain. Therefore, we will throw an error if
> a user tries to use an row-time stream join with late data handling.
> 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)