twalthr commented on a change in pull request #6776: [FLINK-9715][table] 
Support temporal join with event time
URL: https://github.com/apache/flink/pull/6776#discussion_r221595631
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.lang.{Long => JLong}
+import java.util
+import java.util.Comparator
+
+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.runtime.state.{VoidNamespace, VoidNamespaceSerializer}
+import org.apache.flink.streaming.api.SimpleTimerService
+import org.apache.flink.streaming.api.operators._
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.table.api.{StreamQueryConfig, TableException}
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * This operator works by keeping on the state collection of probe and build 
records to process
+  * on next watermark. The idea is that between watermarks we are collecting 
those elements
+  * and once we are sure that there will be no updates we emit the correct 
result and clean up the
+  * state.
+  *
+  * Cleaning up the state drops all of the "old" values from the probe side, 
where "old" is defined
+  * as older then the current watermark. Build side is also cleaned up in the 
similar fashion,
+  * however we always keep at least one record - the latest one - even if it's 
past the last
+  * watermark.
+  *
+  * One more trick is how the emitting results and cleaning up is triggered. 
It is achieved
+  * by registering timers for the keys. We could register a timer for every 
probe and build
+  * side element's event time (when watermark exceeds this timer, that's when 
we are emitting and/or
+  * cleaning up the state). However this would cause huge number of registered 
timers. For example
+  * with following evenTimes of probe records accumulated: {1, 2, 5, 8, 9}, if 
we
+  * had received Watermark(10), it would trigger 5 separate timers for the 
same key. To avoid that
+  * we always keep only one single registered timer for any given key, 
registered for the minimal
+  * value. Upon triggering it, we process all records with event times older 
then currentWatermark.
+  */
+class TemporalRowtimeJoin(
+    leftType: TypeInformation[Row],
+    rightType: TypeInformation[Row],
+    genJoinFuncName: String,
+    genJoinFuncCode: String,
+    queryConfig: StreamQueryConfig,
+    leftTimeAttribute: Int,
+    rightTimeAttribute: Int)
+  extends AbstractStreamOperator[CRow]
+  with TwoInputStreamOperator[CRow, CRow, CRow]
+  with Triggerable[Any, VoidNamespace]
+  with Compiler[FlatJoinFunction[Row, Row, Row]]
+  with Logging {
+
+  validateEqualsHashCode("join", leftType)
+  validateEqualsHashCode("join", rightType)
+
+  private val netLeftIndexStateName = "next-index"
+  private val leftStateName = "left"
+  private val rightStateName = "right"
+  private val registeredTimerStateName = "timer"
+  private val probteTimersStateName = "probe-timers"
+
+  private val rightRowtimeComparator = new 
RowtimeComparator(rightTimeAttribute)
+
+  /**
+    * Incremental index generator for `leftState`'s keys.
+    */
+  private var nextLeftIndex: ValueState[JLong] = _
+
+  /**
+    * This could have been a MultiMap indexed by rowtime, but we have to 
handle rows with
+    * duplicated rowtimes. We can not use List, because we need efficient 
deletes of the oldest
+    * rows.
+    *
+    * TODO: this could be OrderedMultiMap[Jlong, Row] indexed by row's 
timestamp, to avoid
+    * full map traversals (if we have lots of rows on the state that exceed 
`currentWatermark`).
+    */
+  private var leftState: MapState[JLong, Row] = _
+
+  /**
+    * TODO: having `rightState` as an OrderedMapState would allow us to avoid 
sorting cost
+    * once per watermark
+    */
+  private var rightState: MapState[JLong, Row] = _
+
+  private var registeredTimer: ValueState[JLong] = _ // JLong for correct 
handling of default null
+
+  private var cRowWrapper: CRowWrappingCollector = _
+  private var collector: TimestampedCollector[CRow] = _
+  private var timerService: SimpleTimerService = _
+
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  override def open(): Unit = {
+    val clazz = compile(
+      getRuntimeContext.getUserCodeClassLoader,
+      genJoinFuncName,
+      genJoinFuncCode)
+
+    joinFunction = clazz.newInstance()
+
+    nextLeftIndex = getRuntimeContext.getState(
+      new ValueStateDescriptor[JLong](netLeftIndexStateName, 
BasicTypeInfo.LONG_TYPE_INFO))
+    leftState = getRuntimeContext.getMapState(
+      new MapStateDescriptor[JLong, Row](leftStateName, 
BasicTypeInfo.LONG_TYPE_INFO, leftType))
+    rightState = getRuntimeContext.getMapState(
+      new MapStateDescriptor[JLong, Row](rightStateName, 
BasicTypeInfo.LONG_TYPE_INFO, rightType))
+    registeredTimer = getRuntimeContext.getState(
+      new ValueStateDescriptor[JLong](registeredTimerStateName, 
BasicTypeInfo.LONG_TYPE_INFO))
+
+    collector = new TimestampedCollector[CRow](output)
+    cRowWrapper = new CRowWrappingCollector()
+    cRowWrapper.out = collector
+
+    val internalTimerService = getInternalTimerService(
+      probteTimersStateName,
+      VoidNamespaceSerializer.INSTANCE,
+      this)
+
+    timerService = new SimpleTimerService(internalTimerService)
+  }
+
+  override def processElement1(element: StreamRecord[CRow]): Unit = {
+    if (!element.getValue.change) {
+      throw new TableException(
+        s"${classOf[TemporalRowtimeJoin].getSimpleName} does not support 
retractions on the " +
+          s"left side.")
+    }
+
+    leftState.put(getNextLeftIndex, element.getValue.row)
+    maybeRegisterTimer(getLeftTime(element.getValue.row)) // Timer to emit and 
clean up the state
+  }
+
+  override def processElement2(element: StreamRecord[CRow]): Unit = {
+    if (!element.getValue.change) {
+      throw new TableException(
+        s"${classOf[TemporalRowtimeJoin].getSimpleName} does not support 
retractions on the" +
+          s"right side.")
+    }
+
+    val rowTime = getRightTime(element.getValue.row)
+    rightState.put(rowTime, element.getValue.row)
+    maybeRegisterTimer(rowTime) // Timer to clean up the state
+  }
+
+  private def maybeRegisterTimer(timestamp: Long): Unit = {
+    val currentRegisteredTimer = registeredTimer.value()
+    if (currentRegisteredTimer == null) {
+      registerTimer(timestamp)
+    }
+    else if (currentRegisteredTimer != null && currentRegisteredTimer > 
timestamp) {
+      timerService.deleteEventTimeTimer(currentRegisteredTimer)
+      registerTimer(timestamp)
+    }
+  }
+
+  private def registerTimer(timestamp: Long): Unit = {
+    registeredTimer.update(timestamp)
+    timerService.registerEventTimeTimer(timestamp)
+  }
+
+  override def onProcessingTime(timer: InternalTimer[Any, VoidNamespace]): 
Unit = {
+    throw new IllegalStateException("This should never happen")
+  }
+
+  override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = {
+    registeredTimer.clear()
+    val lastUnprocessedTime = 
emitResultAndCleanUpState(timerService.currentWatermark())
+    lastUnprocessedTime.foreach(registerTimer)
+  }
+
+  /**
+    * @return a row time of the oldest unprocessed probe record or None, if 
all records have been
+    *         processed.
+    */
+  private def emitResultAndCleanUpState(timerTimestamp: Long): Option[Long] = {
+    val rightRowsSorted = getRightRowsSorted(rightRowtimeComparator)
+    var lastUnprocessedTime: Option[Long] = None
+
+    val leftIterator = leftState.entries().iterator()
+    while (leftIterator.hasNext) {
+      val leftEntry = leftIterator.next()
+      val leftRow = leftEntry.getValue
+      val leftTime = getLeftTime(leftRow)
+
+      if (leftTime <= timerTimestamp) {
+        val rightRowIndex = latestRightRowToJoin(rightRowsSorted, leftTime)
+
+        if (rightRowIndex >= 0) {
+          val rightRow = rightRowsSorted.get(rightRowIndex)
+
+          cRowWrapper.setChange(true)
 
 Review comment:
   Set the change only once.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to