[
https://issues.apache.org/jira/browse/FLINK-6233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16171860#comment-16171860
]
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_r139723011
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
---
@@ -131,340 +116,308 @@ class TimeBoundedStreamInnerJoin(
// 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)
+ 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)
+ 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])
+ new ValueStateDescriptor[Long](timeIndicator +
"InnerJoinLeftTimerState", classOf[Long])
leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
val rightTimerStateDesc: ValueStateDescriptor[Long] =
- new ValueStateDescriptor[Long](timeIndicator +
"InnerJoinRightTimerState",
- classOf[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
- *
+ * Process rows from the left stream.
*/
override def processElement1(
- cRowValue: CRow,
- ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
- out: Collector[CRow]): Unit = {
- val timeForRecord: Long = getTimeForRecord(ctx, cRowValue, true)
- getCurrentOperatorTime(ctx)
+ cRowValue: CRow,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow]): Unit = {
+ updateOperatorTime(ctx)
+ val rowTime: Long = getTimeForLeftStream(ctx, cRowValue)
+ val oppositeLowerBound: Long = rowTime - rightRelativeSize
+ val oppositeUpperBound: Long = rowTime + leftRelativeSize
processElement(
cRowValue,
- timeForRecord,
+ rowTime,
ctx,
out,
leftOperatorTime,
+ oppositeLowerBound,
+ oppositeUpperBound,
rightOperatorTime,
rightTimerState,
leftCache,
rightCache,
- true
+ leftRow = 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
- *
+ * Process rows from the right stream.
*/
override def processElement2(
- cRowValue: CRow,
- ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
- out: Collector[CRow]): Unit = {
- val timeForRecord: Long = getTimeForRecord(ctx, cRowValue, false)
- getCurrentOperatorTime(ctx)
+ cRowValue: CRow,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow]): Unit = {
+ updateOperatorTime(ctx)
+ val rowTime: Long = getTimeForRightStream(ctx, cRowValue)
+ val oppositeLowerBound: Long = rowTime - leftRelativeSize
+ val oppositeUpperBound: Long = rowTime + rightRelativeSize
processElement(
cRowValue,
- timeForRecord,
+ rowTime,
ctx,
out,
rightOperatorTime,
+ oppositeLowerBound,
+ oppositeUpperBound,
leftOperatorTime,
leftTimerState,
rightCache,
leftCache,
- false
+ leftRow = 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
+ * Put a row from the input stream into the cache and iterate the
opposite cache to
+ * output join results meeting the 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)
- }
+ cRowValue: CRow,
+ timeForRow: Long,
+ ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+ out: Collector[CRow],
+ myWatermark: Long,
+ oppositeLowerBound: Long,
+ oppositeUpperBound: Long,
+ oppositeWatermark: Long,
+ oppositeTimeState: ValueState[Long],
+ rowListCache: MapState[Long, JList[Row]],
+ oppositeCache: MapState[Long, JList[Row]],
+ leftRow: Boolean): Unit = {
+ cRowWrapper.out = out
+ val row = cRowValue.row
+ if (!checkRowOutOfDate(timeForRow, myWatermark)) {
+ // Put the row into the cache for later use.
+ var rowList = rowListCache.get(timeForRow)
+ if (null == rowList) {
+ rowList = new ArrayList[Row](1)
+ }
+ rowList.add(row)
+ rowListCache.put(timeForRow, rowList)
+ // Register a timer on THE OPPOSITE STREAM to remove rows from the
cache once they are
+ // expired.
+ if (oppositeTimeState.value == 0) {
+ registerCleanUpTimer(
+ ctx, timeForRow, oppositeWatermark, oppositeTimeState, leftRow,
firstTimer = 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.
- } else if (oppositeTime >= oppositeLowerBound && oppositeTime <=
oppositeUpperBound) {
- val oppositeRows = oppositeEntry.getValue
- var i = 0
- if (leftRecord) {
- while (i < oppositeRows.size) {
- joinFunction.join(record, oppositeRows.get(i), cRowWrapper)
- i += 1
- }
- } else {
- while (i < oppositeRows.size) {
- joinFunction.join(oppositeRows.get(i), record, cRowWrapper)
- i += 1
- }
+ // Join the row with rows from the opposite stream.
+ val oppositeIterator = oppositeCache.iterator()
+ while (oppositeIterator.hasNext) {
+ val oppositeEntry = oppositeIterator.next
+ val oppositeTime = oppositeEntry.getKey
+ if (oppositeTime >= oppositeLowerBound && oppositeTime <=
oppositeUpperBound) {
+ val oppositeRows = oppositeEntry.getValue
+ var i = 0
+ if (leftRow) {
+ while (i < oppositeRows.size) {
+ joinFunction.join(row, oppositeRows.get(i), cRowWrapper)
+ i += 1
+ }
+ } else {
+ while (i < oppositeRows.size) {
+ joinFunction.join(oppositeRows.get(i), row, cRowWrapper)
+ i += 1
}
- } else if (oppositeTime > oppositeUpperBound) {
- //TODO If the keys are ordered, can we break here?
}
}
- } else {
- //TODO Need some extra logic here?
- LOG.warn(s"$record is out-of-date.")
+ // We could do the short-cutting optimization here once we get a
state with ordered keys.
}
}
+ // We need to deal with the late data in the future.
--- End diff --
Yes, I agree. That would be a bigger effort though, because it should
involve the other operators as well. So far we do not provide any metrics for
the relational operators but it would make sense to do so.
> 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)