flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9715) Support versioned joins with event time
Date Tue, 16 Oct 2018 09:20:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-9715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651379#comment-16651379
] 

ASF GitHub Bot commented on FLINK-9715:
---------------------------------------

kl0u 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_r225459775
 
 

 ##########
 File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
 ##########
 @@ -0,0 +1,339 @@
+/*
+ * 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
+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 org.apache.flink.util.Preconditions.checkState
+
+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 or equal
to
+  * 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 NEXT_LEFT_INDEX_STATE_NAME = "next-index"
+  private val LEFT_STATE_NAME = "left"
+  private val RIGHT_STATE_NAME = "right"
+  private val REGISTERED_TIMER_STATE_NAME = "timer"
+  private val TIMERS_STATE_NAME = "timers"
+
+  private val rightRowtimeComparator = new RowtimeComparator(rightTimeAttribute)
+
+  /**
+    * Incremental index generator for `leftState`'s keys.
+    */
+  private var nextLeftIndex: ValueState[JLong] = _
+
+  /**
+    * Mapping from artificial row index (generated by `nextLeftIndex`) into the left side
`Row`.
+    * We can not use List to accumulate Rows, 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] = _
+
+  /**
+    * Mapping from timestamp to right side `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](NEXT_LEFT_INDEX_STATE_NAME, BasicTypeInfo.LONG_TYPE_INFO))
+    leftState = getRuntimeContext.getMapState(
+      new MapStateDescriptor[JLong, Row](LEFT_STATE_NAME, BasicTypeInfo.LONG_TYPE_INFO, leftType))
+    rightState = getRuntimeContext.getMapState(
+      new MapStateDescriptor[JLong, Row](RIGHT_STATE_NAME, BasicTypeInfo.LONG_TYPE_INFO,
rightType))
+    registeredTimer = getRuntimeContext.getState(
+      new ValueStateDescriptor[JLong](REGISTERED_TIMER_STATE_NAME, BasicTypeInfo.LONG_TYPE_INFO))
+
+    collector = new TimestampedCollector[CRow](output)
+    cRowWrapper = new CRowWrappingCollector()
+    cRowWrapper.out = collector
+    cRowWrapper.setChange(true)
+
+    val internalTimerService = getInternalTimerService(
+      TIMERS_STATE_NAME,
+      VoidNamespaceSerializer.INSTANCE,
+      this)
+
+    timerService = new SimpleTimerService(internalTimerService)
+  }
+
+  override def processElement1(element: StreamRecord[CRow]): Unit = {
+    checkState(
+      element.getValue.change,
+      "Retractions are not supported by [{}]. " +
+        "If this can happen it should be validated during planning!",
+      classOf[TemporalRowtimeJoin].getSimpleName)
+
+    leftState.put(getNextLeftIndex, element.getValue.row)
+    registerSmallestTimer(getLeftTime(element.getValue.row)) // Timer to emit and clean up
the state
+  }
+
+  override def processElement2(element: StreamRecord[CRow]): Unit = {
+    checkState(
+      element.getValue.change,
+      "Retractions are not supported by [{}]. " +
+        "If this can happen it should be validated during planning!",
+      classOf[TemporalRowtimeJoin].getSimpleName)
+
+    val rowTime = getRightTime(element.getValue.row)
+    rightState.put(rowTime, element.getValue.row)
+    registerSmallestTimer(rowTime) // Timer to clean up the state
+  }
+
+  private def registerSmallestTimer(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())
+    if (lastUnprocessedTime < Long.MaxValue) {
+      registerTimer(lastUnprocessedTime)
+    }
+  }
+
+  /**
+    * @return a row time of the oldest unprocessed probe record or Long.MaxValue, if all
records
+    *         have been processed.
+    */
+  private def emitResultAndCleanUpState(timerTimestamp: Long): Long = {
+    val rightRowsSorted = getRightRowsSorted(rightRowtimeComparator)
+    var lastUnprocessedTime = Long.MaxValue
+
+    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)
+
+          joinFunction.join(leftRow, rightRow, cRowWrapper)
+        }
+        leftIterator.remove()
+      }
+      else {
+        lastUnprocessedTime = Math.min(lastUnprocessedTime, leftTime)
+      }
+    }
+
+    cleanUpState(timerTimestamp, rightRowsSorted)
+    lastUnprocessedTime
+  }
+
+  /**
+    * Removes all right entries older then the watermark, except the latest one. For example
with:
+    * rightState = [1, 5, 9]
+    * and
+    * watermark = 6
+    * we can not remove "5" from rightState, because left elements with rowtime of 7 or 8
could
+    * be joined with it later
+    */
+  private def cleanUpState(timerTimestamp: Long, rightRowsSorted: util.List[Row]) = {
+    var i = 0
+    while (i < firstIndexToKeep(timerTimestamp, rightRowsSorted)) {
+      val rightTime = getRightTime(rightRowsSorted.get(i))
+      rightState.remove(rightTime)
+      i += 1
+    }
+  }
+
+  private def firstIndexToKeep(timerTimestamp: Long, rightRowsSorted: util.List[Row]): Int
= {
+    val firstIndexNewerThenTimer =
+      indexOf[Row](rightRowsSorted, row => getRightTime(row) > timerTimestamp)
+
+    if (firstIndexNewerThenTimer < 0) {
+      rightRowsSorted.size() - 1
+    }
+    else {
+      firstIndexNewerThenTimer - 1
+    }
+  }
+
+  private def indexOf[T](list: util.List[T], predicate: T => Boolean): Int = {
+    val iter = list.listIterator
+    while (iter.hasNext) {
+      if (predicate(iter.next)) {
+        return iter.previousIndex
+      }
+    }
+    -1
+  }
+
+  /**
+    * Binary search `rightRowsSorted` to find the latest right row to join with `leftTime`.
+    * Latest means a right row with largest time that is still smaller or equal to `leftTime`.
+    *
+    * @return index of such element. If such row was not found (either `rightRowsSorted`
is empty
+    *         or all `rightRowsSorted` are are newer) return -1.
+    */
+  private def latestRightRowToJoin(rightRowsSorted: util.List[Row], leftTime: Long): Int
= {
+    latestRightRowToJoin(rightRowsSorted, 0, rightRowsSorted.size - 1, leftTime)
+  }
+
+  private def latestRightRowToJoin(
 
 Review comment:
   This method it seems that it could return the element itself and not its index. This would
save one state access.

----------------------------------------------------------------
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:
users@infra.apache.org


> Support versioned joins with event time
> ---------------------------------------
>
>                 Key: FLINK-9715
>                 URL: https://issues.apache.org/jira/browse/FLINK-9715
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API &amp; SQL
>    Affects Versions: 1.5.0
>            Reporter: Piotr Nowojski
>            Assignee: Piotr Nowojski
>            Priority: Major
>              Labels: pull-request-available
>
> Queries like:
> {code:java}
> SELECT 
>   o.amount * r.rate 
> FROM 
>   Orders AS o, 
>   LATERAL TABLE (Rates(o.rowtime)) AS r 
> WHERE o.currency = r.currency{code}
> should work with event time



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message