You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by xccui <gi...@git.apache.org> on 2017/08/30 15:41:40 UTC

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

GitHub user xccui opened a pull request:

    https://github.com/apache/flink/pull/4625

    [FLINK-6233] [table] Support time-bounded stream inner join in the SQL API

    ## What is the purpose of the change
    
    This PR aims add an implementation of the time-bounded stream inner join for both proctime and rowtime in the SQL API. For example, ``SELECT * from L, R WHERE L.pid = R.pid AND L.time between R.time + X and R.time + Y``. A design document for this problem can be found [here](http://goo.gl/VW5Gpd).
    
    
    ## Brief change log
    
      - I fill the missing part of the compiling stage for the rowtime stream inner join.
      - Some logics are added to the `WindowJoinUtil` to extract the rowtime indices.
      - A general `TimeBoundedStreamInnerJoin` is provided.
      - To test the new join function, I add a `TimeBoundedJoinExample` and some new tests to the `JoinHarnessTest`.
    
    ## Verifying this change
    
    This change added tests to the existing JoinHarnessTest.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (**no**)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (**no**)
      - The serializers: (**no**)
      - The runtime per-record code paths (performance sensitive): (**no**)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (**no**)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (**yes**)
      - If yes, how is the feature documented? (**not documented yet**)

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/xccui/flink FLINK-6233

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4625.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4625
    
----
commit c79588b134a1270956a6d32b7a0a13ff4e3f483d
Author: Xingcan Cui <xi...@gmail.com>
Date:   2017-08-30T05:57:38Z

    [FLINK-6233] [table] Support rowtime inner equi-join between two streams in the SQL API

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r141831721
  
    --- 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)) {
    --- End diff --
    
    There is no problem with the unified semantics for batch and streaming. In fact, handling as much late data as possible will result in closer results of streaming and batch queries. 
    
    You can think of a batch table as a streaming table that does not provide watermark except for one `Long.MAX_VALUE` watermark after all rows have been shipped. Hence, batch tables have no late data and the more (late) data a streaming query processes the closer its result is to the result of an equivalent batch query.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
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. 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137123724
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -232,4 +232,164 @@ class JoinHarnessTest extends HarnessTestBase{
         testHarness.close()
       }
     
    +  /** a.proctime >= b.proctime - 10 and a.proctime <= b.proctime + 20 **/
    +  @Test
    +  def testNewNormalProcTimeJoin() {
    --- End diff --
    
    We would also need harness tests for the event-time join.
    In addition we need:
    - plan translation tests (see `org.apache.flink.table.api.stream.sql.JoinTest`)
    - some end-to-end ITCases (see `org.apache.flink.table.runtime.stream.sql.JoinITCase` but with result validation)
    
    Moreover, the processing time restriction for windowed joins should be removed from the documentation.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143666687
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    Thanks for the explanation. Let's keep the current implementation. 


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Hi @fhueske, the PR has been updated. However, there are still some unfinished tasks, e.g., optimise the data caching and cleaning up policies and distinguish the `<` and `<=`signs. I want to leave them as future works. What do you think?
    
    BTW, I find most of the recent PRs are failed on the build jobs with `TEST = "misc"`.  I'm not sure if there exist some problems with the CI. Here is a [build log](https://s3.amazonaws.com/archive.travis-ci.org/jobs/274508987/log.txt?X-Amz-Expires=30&X-Amz-Date=20170912T130101Z&X-Amz-Algorithm=AWS4-HMAC-SHA256&X-Amz-Credential=AKIAJRYRXRSVGNKPKO5A/20170912/us-east-1/s3/aws4_request&X-Amz-SignedHeaders=host&X-Amz-Signature=af6d9a141ef245be9a7393e69c484785340a9bb1f1030e1039d4ed44f0344a42) for this PR.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137117103
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    --- End diff --
    
    can be the `else` case without condition, right?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
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.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142705185
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    --- End diff --
    
    Add a comment: "Check if we need to cache the current row."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139385852
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -57,6 +61,8 @@ class DataStreamWindowJoin(
         with CommonJoin
         with DataStreamRel {
     
    +  lazy val LOG: Logger = LoggerFactory.getLogger(getClass)
    --- End diff --
    
    use `Logging` trait instead of creating logger yourself.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139400962
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    +
    +    val joinProcessFunc = new RowTimeBoundedStreamInnerJoin(
    +      -10, 20, 0, rT, rT, "TestJoinFunction", funcCode, 0, 0)
    +
    +    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[String](1),
    +        new TupleRowKeySelector[String](1),
    +        BasicTypeInfo.STRING_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // Advance
    +    testHarness.processWatermark1(new Watermark(1))
    +    testHarness.processWatermark2(new Watermark(1))
    +
    +    // Test late data
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(1L: JLong, "k1"), true), 0))
    +
    +    assertEquals(0, testHarness.numEventTimeTimers())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +
    +    assertEquals(2, testHarness.numEventTimeTimers())
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(5L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(15L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(20))
    +    testHarness.processWatermark2(new Watermark(20))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(35L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(38))
    +    testHarness.processWatermark2(new Watermark(38))
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(40L: JLong, "k2"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(39L: JLong, "k2"), true), 0))
    +
    +    assertEquals(6, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processWatermark1(new Watermark(61))
    +    testHarness.processWatermark2(new Watermark(61))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    +    expectedOutput.add(new StreamRecord(
    +      CRow(Row.of(2L: JLong, "k1", 2L: JLong, "k1"), true), 0))
    +    expectedOutput.add(new StreamRecord(
    +      CRow(Row.of(5L: JLong, "k1", 2L: JLong, "k1"), true), 0))
    +    expectedOutput.add(new StreamRecord(
    +      CRow(Row.of(5L: JLong, "k1", 15L: JLong, "k1"), true), 0))
    +    expectedOutput.add(new StreamRecord(
    +      CRow(Row.of(35L: JLong, "k1", 15L: JLong, "k1"), true), 0))
    +    expectedOutput.add(new StreamRecord(
    +      CRow(Row.of(40L: JLong, "k2", 39L: JLong, "k2"), true), 0))
    +
    +    val result = testHarness.getOutput
    +    verify(expectedOutput, result, new RowResultSortComparator())
    +    testHarness.close()
    +  }
    +
    +  /** a.rowtime >= b.rowtime - 10 and a.rowtime <= b.rowtime - 7 **/
    +  @Test
    +  def testNegativeRowTimeJoin() {
    +
    +    val joinProcessFunc = new RowTimeBoundedStreamInnerJoin(
    +      -10, -7, 0, rT, rT, "TestJoinFunction", funcCode, 0, 0)
    +
    +    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[String](1),
    +        new TupleRowKeySelector[String](1),
    +        BasicTypeInfo.STRING_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // Advance
    --- End diff --
    
    Add more inline comments.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137013584
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    --- End diff --
    
    Change to `"InnerRowtimeWindowJoin",`?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139434334
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
         }
    +    registerTimer(ctx, !leftRow, cleanupTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRow) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRow = ${rowTime}, cleanupTime should be ${cleanupTime - cleanupDelay}," +
    +      s" but delayed to ${cleanupTime}," +
    +      s" oppositeWatermark = ${oppositeWatermark}")
    +    timerState.update(cleanupTime)
    +    //if cleanupTime <= oppositeWatermark + allowedLateness && firstTimer, we may set the
    +    //  backPressureSuggestion =
    +    //    if (leftRow) (oppositeWatermark + allowedLateness - cleanupTime)
    +    //    else -(oppositeWatermark + allowedLateness - cleanupTime)
       }
     
    -
       /**
         * Called when a registered timer is fired.
    -    * Remove records which are earlier than the expiration time,
    -    * and register a new timer for the earliest remaining records.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
         *
         * @param timestamp the timestamp of the timer
         * @param ctx       the context to register timer or get current time
         * @param out       the collector for returning result values
         */
       override def onTimer(
    -    timestamp: Long,
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    -    out: Collector[CRow]): Unit = {
    -    getCurrentOperatorTime(ctx)
    -    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    -    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    -    //TODO already been emitted by the source.
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
         if (leftTimerState.value == timestamp) {
           val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    -      removeExpiredRecords(
    -        timestamp,
    +      removeExpiredRows(
             rightExpirationTime,
             leftOperatorTime,
             rightCache,
             leftTimerState,
             ctx,
    -        false
    +        removeLeft = false
           )
         }
     
         if (rightTimerState.value == timestamp) {
           val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    -      removeExpiredRecords(
    -        timestamp,
    +      removeExpiredRows(
             leftExpirationTime,
             rightOperatorTime,
             leftCache,
             rightTimerState,
             ctx,
    -        true
    +        removeLeft = true
           )
         }
       }
     
       /**
    -    * Remove the expired records. Register a new timer if the cache still holds records
    +    * Remove the expired rows. Register a new timer if the cache still holds valid rows
         * after the cleaning up.
    +    *
    +    * @param expirationTime    the expiration time for this cache
    +    * @param oppositeWatermark the watermark of the opposite stream
    +    * @param rowCache          the row cache
    +    * @param timerState        timer state for the opposite stream
    +    * @param ctx               the context to register the cleanup timer
    +    * @param removeLeft        whether to remove the left rows
         */
    -  private def removeExpiredRecords(
    -    timerFiringTime: Long,
    -    expirationTime: Long,
    -    oppositeWatermark: Long,
    -    recordCache: MapState[Long, JList[Row]],
    -    timerState: ValueState[Long],
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    -    removeLeft: Boolean): Unit = {
    +  private def removeExpiredRows(
    +      expirationTime: Long,
    +      oppositeWatermark: Long,
    +      rowCache: MapState[Long, JList[Row]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      removeLeft: Boolean): Unit = {
     
    -    val keysIterator = recordCache.keys().iterator()
    +    val keysIterator = rowCache.keys().iterator()
     
         // Search for expired timestamps.
         // If we find a non-expired timestamp, remember the timestamp and leave the loop.
         // This way we find all expired timestamps if they are sorted without doing a full pass.
         var earliestTimestamp: Long = -1L
    -    var recordTime: Long = 0L
    +    var rowTime: Long = 0L
         while (keysIterator.hasNext) {
    -      //TODO The "short-circuit" code was commented, because when using a StateMap with
    -      //TODO unordered keys, the cache will grow indefinitely!
    -      // && earliestTimestamp < 0) {
    -      recordTime = keysIterator.next
    -      if (recordTime <= expirationTime) {
    -        // TODO Not sure if we can remove records directly.
    +      rowTime = keysIterator.next
    +      if (rowTime <= expirationTime) {
             keysIterator.remove()
           } else {
             // We find the earliest timestamp that is still valid.
    -        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    -          earliestTimestamp = recordTime
    +        if (rowTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = rowTime
             }
           }
         }
         // If the cache contains non-expired timestamps, register a new timer.
         // Otherwise clear the states.
         if (earliestTimestamp > 0) {
    -      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +      registerCleanUpTimer(
    +        ctx,
    +        earliestTimestamp,
    --- End diff --
    
    I'd apply the `cleanupDelay` here. The first timer does not need it.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137035608
  
    --- 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
    --- End diff --
    
    this might be understood as both `BETWEEN` predicates could be evaluated at the same time which is not the case.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137100457
  
    --- 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)) {
    --- End diff --
    
    we should directly call `get()` and check for a `null` return value. `contains()` and `get()` would result in two RocksDB accesses of which one can be avoided.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140255052
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.util.{ArrayList, 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.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +  //For delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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 rows from the left stream.
    +    */
    +  override def processElement1(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      leftOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      rightOperatorTime,
    +      rightTimerState,
    +      leftCache,
    +      rightCache,
    +      leftRow = true
    +    )
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      rightOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      leftOperatorTime,
    +      leftTimerState,
    +      rightCache,
    +      leftCache,
    +      leftRow = false
    +    )
    +  }
    +
    +  /**
    +    * 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,
    +      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)
    --- End diff --
    
    Hi @fhueske, that's a good idea for me! We could start from a constant granularity and make it dynamic in the future.
    BTW, I'm not sure if you still remember the following comment in FLINK-6233 😄 
    > To cope with that, I plan to split the "cache window" into continuous static-panes, and casting one to expired as a whole. By doing like that, we may store some extra records, whose time interval is equal to the static span of the panes, but can remove the expired data efficiently.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137227871
  
    --- 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) {
    --- End diff --
    
    I see, but adding a condition at this place adds overhead for the common case that will (or should) always be true. 
    
    I'd rather add a different operator that doesn't emit anything during plan construction.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137032484
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala ---
    @@ -225,7 +233,7 @@ object WindowJoinUtil {
               case t: TimeIndicatorRelDataType =>
                 // time attribute access. Remember time type and side of input
                 val isLeftInput = idx < leftFieldCount
    -            Seq(TimeAttributeAccess(t.isEventTime, isLeftInput))
    +            Seq(TimeAttributeAccess(t.isEventTime, isLeftInput, idx))
    --- End diff --
    
    I would do the subtraction here:
    
    ```
    if (idx < leftFieldCount) {
      Seq(TimeAttributeAccess(t.isEventTime, true, idx))
    } else {
      Seq(TimeAttributeAccess(t.isEventTime, false, idx - leftFieldCount))
    }
    ```



---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137040050
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    --- End diff --
    
    We should rather make this an `abstract` class and add two classes for event and processing time that extend it.
    This will avoid many condition checks at runtime. `getTimeForRecord()` should be changed to an `abstract` method that is implemented by both subclasses.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139430727
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    --- End diff --
    
    `leftOperatorTime = rightOperatorTime` to ensure that both sides have the same time


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139611419
  
    --- 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 --
    
    Totally agree. We don't need to handle late records. However, I think we should provide some runtime statistics about the late data. That could help the user know more about the data/the watermark assigner in use. What do you think?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139426723
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
         }
    +    registerTimer(ctx, !leftRow, cleanupTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRow) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRow = ${rowTime}, cleanupTime should be ${cleanupTime - cleanupDelay}," +
    +      s" but delayed to ${cleanupTime}," +
    +      s" oppositeWatermark = ${oppositeWatermark}")
    +    timerState.update(cleanupTime)
    +    //if cleanupTime <= oppositeWatermark + allowedLateness && firstTimer, we may set the
    --- End diff --
    
    remove comments


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139410699
  
    --- 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) {
    --- End diff --
    
    I think it make sense to move the implementation to `processElement1()` and `processElement2()` because we have to check for `leftRow` again. We might even save a few lines of code due to the long list of method parameters. Moreover, we save a method call and several condition checks.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137146303
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    +            rowTimeInnerJoinFunc,
    +            rowTimeInnerJoinFunc.getMaxOutputDelay)
    --- End diff --
    
    I think the "watermark delay" is considered from the operator level while the "output delay" is named from the function level. So how about keep this name?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139401341
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    +
    +    val joinProcessFunc = new RowTimeBoundedStreamInnerJoin(
    +      -10, 20, 0, rT, rT, "TestJoinFunction", funcCode, 0, 0)
    +
    +    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[String](1),
    +        new TupleRowKeySelector[String](1),
    +        BasicTypeInfo.STRING_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // Advance
    +    testHarness.processWatermark1(new Watermark(1))
    +    testHarness.processWatermark2(new Watermark(1))
    +
    +    // Test late data
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(1L: JLong, "k1"), true), 0))
    +
    +    assertEquals(0, testHarness.numEventTimeTimers())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +
    +    assertEquals(2, testHarness.numEventTimeTimers())
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(5L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(15L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(20))
    +    testHarness.processWatermark2(new Watermark(20))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(35L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(38))
    +    testHarness.processWatermark2(new Watermark(38))
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(40L: JLong, "k2"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(39L: JLong, "k2"), true), 0))
    +
    +    assertEquals(6, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processWatermark1(new Watermark(61))
    +    testHarness.processWatermark2(new Watermark(61))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    --- End diff --
    
    We should test the boundaries of all join windows (left/right, lower/upper) to ensure we don't have off-by-one bugs.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137031980
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala ---
    @@ -115,10 +118,15 @@ object WindowJoinUtil {
           case _ =>
             Some(otherPreds.reduceLeft((l, r) => RelOptUtil.andJoinFilters(rexBuilder, l, r)))
         }
    -
    -    val bounds = Some(WindowBounds(timePreds.head.isEventTime, leftLowerBound, leftUpperBound))
    -
    -    (bounds, remainCondition)
    +    if (timePreds.head.leftInputOnLeftSide) {
    --- End diff --
    
    Please replace the condition as follows:
    ```
      val bounds = if (timePreds.head.leftInputOnLeftSide) {
        Some(WindowBounds(
          timePreds.head.isEventTime,
          leftLowerBound,
          leftUpperBound,
          timePreds.head.leftTimeIdx,
          timePreds.head.rightTimeIdx))
      } else {
        Some(WindowBounds(
          timePreds.head.isEventTime,
          leftLowerBound,
          leftUpperBound,
          timePreds.head.rightTimeIdx,
          timePreds.head.leftTimeIdx))
      }
    
      (bounds, remainCondition)
    ```
    
    We should not get the right index from the second time predicate because the tables might be accessed in inverse order there (we would need to check `leftInputOnLeftSide` again).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140201035
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.util.{ArrayList, 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.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +  //For delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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 rows from the left stream.
    +    */
    +  override def processElement1(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      leftOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      rightOperatorTime,
    +      rightTimerState,
    +      leftCache,
    +      rightCache,
    +      leftRow = true
    +    )
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      rightOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      leftOperatorTime,
    +      leftTimerState,
    +      rightCache,
    +      leftCache,
    +      leftRow = false
    +    )
    +  }
    +
    +  /**
    +    * 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,
    +      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)
    --- End diff --
    
    Just had a discussion about this with a colleague. 
    
    He suggested to round the timestamp to reduce the number of keys in the state backend. This would also mean that we can directly address all keys (because we can compute them) that we need to join with and don't need to iterate over all keys. Clean-up would also work without full traversal.
    
    However, when joining we would need to check again the window condition because a list might contain records that have the same rounded timestamp but are outside of the window. For that we need to store the rows with timestamps (at least for proctime, rowtime has the key already in the row).
    
    This design should give much better performance because we only access relevant keys. However, we would need to decide for a key granularity. I think a few seconds (10 seconds) could be a good starting point.
    
    What do you think about this @xccui?


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Merging


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139727416
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForLeftStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    --- End diff --
    
    To be honest, I would not put too much effort into the processing time case, especially not if it affects the performance of event-time processing. Processing time is non-deterministic anyway. The reason I brought this up is because I wasn't sure of the side effects if the the row proctime > operator time. If this is not an issue, we can keep it like this. 
    
    Otherwise, the easiest solution would be to just add a comment to the invocations of `updateOperatorTime` that this call must be the first call in all processing methods (`processElement()`, `onTimer()`). Since this is just internal API, this should be fine.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142679463
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala ---
    @@ -105,6 +104,8 @@ class DataStreamWindowJoinRule
           windowBounds.get.isEventTime,
           windowBounds.get.leftLowerBound,
           windowBounds.get.leftUpperBound,
    +      windowBounds.get.leftTimeIdx,
    +      windowBounds.get.rightTimeIdx,
           remainCondition,
    --- End diff --
    
    The `remainCondition` must include the equi-join predicates to ensure that the join condition is correctly evaluated for `null` values (see FLINK-7755 for details).
    
    To solve this, I'd suggest to call `WindowJoinUtil.extractWindowBoundsFromPredicate` with `join.getCondition` instead of `joinInfo.getRemaining(join.getCluster.getRexBuilder)`.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137168799
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    --- End diff --
    
    I am thinking if the `rightState.value` could be equal with the `leftState.value`.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142706610
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    I don't think we need this check. 
    A source function emits a `Long.MaxValue` watermark once it read all its data. Hence we won't get any more rows, if the operator time is `Long.MaxValue`. So the state won't grow any further and the query is going to terminate soon.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137225205
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala ---
    @@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
     
         if (windowBounds.isDefined) {
           if (windowBounds.get.isEventTime) {
    -        // we cannot handle event-time window joins yet
    -        false
    +        val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +          .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
    +
    +        !remainingPredsAccessTime && !procTimeAttrInOutput
    --- End diff --
    
    No, the problem is that these rowtime attributes after a proc-time join won't be aligned with the watermarks anymore. We would need to hold back watermarks based on the data in the caches and not based on the window boundaries. 
    
    Keeping the proctime attributes is fine because they are not bound to watermarks.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143208392
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    --- End diff --
    
    Because it's the previous expiration time (either being calculated in `processElement()` or `onTime()`) that really decides the state of the current cache. Say `prevMinTime` and `prevExpirationTime` are the contained minimum time and the expiration time of the cache **before** processing the current row. The condition here should be `prevMinTime < rightQualifiedUpperBound`. However, since the `prevMinTime` is hard to be calculated, I use `prevExpirationTime` as an approximation of `prevMinTime` here. Actually,  `prevExpirationTime <= prevMinTime` is always `true`, thus we won't miss any opportunities to join.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143400052
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +229,50 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
    +      joinFunctionName: String,
    +      joinFunctionCode: String,
    +      leftKeys: Array[Int],
    +      rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val rowTimeInnerJoinFunc = new RowTimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness = 0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx)
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    --- End diff --
    
    That's a good point. The motivation for this restriction is to prevent nested-loop joins for the batch execution. 
    In the streaming window join case, no equi-join keys would result in a single thread execution which is not efficient either (could be parallelized by broadcasting one of the inputs but without statistics this is quite risky) but not as bad as a full nested-loop join because we can bound the computation due to the window predicates.
    
    We could add a boolean flag to the constructor of `FlinkLogicalJoinConverter` to allow cross joins. Right now the set of logical optimization rules is shared by DataSet and DataStream. I'd keep the joint rule set for now and just add an additional rule by overriding `getLogicalOptRuleSet` in `StreamExecutionEnvironment`. 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137096261
  
    --- 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,
    --- End diff --
    
    please indent the method parameters by 4 spaces to separate them from the following main body, i.e.,
    ```
    override def processElement1(
        cRowValue: CRow,
        ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
        out: Collector[CRow]): Unit = {
    
      val timeForRecord: Long = getTimeForRecord(ctx, cRowValue, isLeft = true)
      updateOperatorTime(ctx)
      processElement(
        ...
      )
    }
    ```


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140262200
  
    --- 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)) {
    --- End diff --
    
    Thanks for your thoughts. I understand your motivation to either emit complete results for a row or none but I think it is fine to emit partial results. My reasoning is the following:
    
    1. we should try to produce results which are as close to the original semantics as possible
    2. being late (later than the allowedLateness) does not necessarily mean that we missed matching results. Maybe there were no matching records in the matched time frame, so we could still produce exact results.
    3. By not storing a row in the cache, the join result becomes more incomplete. Rows of the other stream that would have produced a complete result will only produce partial results (which we wanted to avoid by not adding it to state).
    
    So, I'd rather unnest the conditions.


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Thanks for the update @xccui. I'll have a look in the next days.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139633281
  
    --- 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) {
    --- End diff --
    
    If I understand correctly, you meant to remove the `processElement()` method and move the implementation to `processElement1()` and `processElement2()`, right?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139849018
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = {
    +    timeForRow <= watermark - allowedLateness
    +  }
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    --- End diff --
    
    Totally understand 😄 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142762460
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    +      operatorTime - relativeSize - allowedLateness - 1
    +    } else {
    +      Long.MaxValue
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up rows in a specified time.
    +    *
    +    * @param ctx        the context to register timer
    +    * @param rowTime    time for the input row
    +    * @param timerState stores the timestamp for the next timer
    +    * @param leftRow    whether this row comes from the left stream
    +    */
    +  private def registerCleanUpTimer(
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
    +    }
    +    registerTimer(ctx, cleanupTime)
    +    timerState.update(cleanupTime)
    +  }
    +
    +  /**
    +    * Remove the expired rows. Register a new timer if the cache still holds valid rows
    +    * after the cleaning up.
    +    *
    +    * @param expirationTime the expiration time for this cache
    +    * @param rowCache       the row cache
    +    * @param timerState     timer state for the opposite stream
    +    * @param ctx            the context to register the cleanup timer
    +    * @param removeLeft     whether to remove the left rows
    +    */
    +  private def removeExpiredRows(
    +      expirationTime: Long,
    +      rowCache: MapState[Long, JList[Row]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = rowCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    --- End diff --
    
    Update comment. We remove all expired keys and do not leave the loop early. Hence, we do a full pass over the state.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139401824
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    +
    +    val joinProcessFunc = new RowTimeBoundedStreamInnerJoin(
    +      -10, 20, 0, rT, rT, "TestJoinFunction", funcCode, 0, 0)
    +
    +    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[String](1),
    +        new TupleRowKeySelector[String](1),
    +        BasicTypeInfo.STRING_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // Advance
    +    testHarness.processWatermark1(new Watermark(1))
    +    testHarness.processWatermark2(new Watermark(1))
    +
    +    // Test late data
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(1L: JLong, "k1"), true), 0))
    +
    +    assertEquals(0, testHarness.numEventTimeTimers())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(2L: JLong, "k1"), true), 0))
    +
    +    assertEquals(2, testHarness.numEventTimeTimers())
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(5L: JLong, "k1"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(15L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(20))
    +    testHarness.processWatermark2(new Watermark(20))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(35L: JLong, "k1"), true), 0))
    +
    +    testHarness.processWatermark1(new Watermark(38))
    +    testHarness.processWatermark2(new Watermark(38))
    +
    +    testHarness.processElement1(new StreamRecord[CRow](
    +      CRow(Row.of(40L: JLong, "k2"), true), 0))
    +    testHarness.processElement2(new StreamRecord[CRow](
    +      CRow(Row.of(39L: JLong, "k2"), true), 0))
    +
    +    assertEquals(6, testHarness.numKeyedStateEntries())
    +
    +    testHarness.processWatermark1(new Watermark(61))
    +    testHarness.processWatermark2(new Watermark(61))
    +
    +    assertEquals(4, testHarness.numKeyedStateEntries())
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    --- End diff --
    
    Add multiple rows for the same key and time to validate that this case is correctly handled. It might make sense to add another string field to the data with a unique value ("left1", ...) to make the input and output records easier to compare.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140645274
  
    --- 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)) {
    --- End diff --
    
    Hi @fhueske, I got another concern that if we produce partial results, will that affect the  batch/stream SQL consolidation? Besides, the semantics of the watermark will not be hold there. I suggest to consult more people and make this decision carefully.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142703252
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    --- End diff --
    
    rm `;`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137095404
  
    --- 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",
    --- End diff --
    
    Please move all method arguments (incl. the first) into an own line if you have to break them up, i.e.,
    ```
    val leftStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
          new MapStateDescriptor[Long, JList[Row]](
            timeIndicator + "InnerJoinLeftCache",
            BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]],
            leftListTypeInfo)
    ```


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139436010
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForLeftStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForRightStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    --- End diff --
    
    should be `rightOperatorTime` to ensure that both are the same?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137163327
  
    --- 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) {
    --- End diff --
    
    If `relativeWindowSize <= 0`, the whole function will be executed without any problem, but no results can be produced. This check was added as an optimization. However, I think we should not reject this case (a warning is enough). After all, it's an improper condition instead of an error. What do you think?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137225735
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    --- End diff --
    
    I think it is fine to call it window join. The join predicate effectively defines a window for each row a join window on the opposite stream. In fact, these windows are more similar to `OVER` windows than group windows such as `TUMBLE` or `HOP`.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137010384
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala ---
    @@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
     
         if (windowBounds.isDefined) {
           if (windowBounds.get.isEventTime) {
    -        // we cannot handle event-time window joins yet
    -        false
    +        val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +          .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
    +
    +        !remainingPredsAccessTime && !procTimeAttrInOutput
    --- End diff --
    
    change to `!remainingPredsAccessTime`. It is OK to keep processing time attributes in the join output.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142681690
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -152,19 +176,40 @@ class DataStreamWindowJoin(
         }
       }
     
    -  def createProcTimeInnerJoinFunction(
    +  def createEmptyInnerJoin(
    --- End diff --
    
    please add the return type for the method `DataStream[CRow]`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137040311
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        return if (isLeft) {
    +          record.row.getField(leftTimeIdx).asInstanceOf[Long]
    +        } else {
    +          record.row.getField(rightTimeIdx).asInstanceOf[Long];
    --- End diff --
    
    rm `;`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139399103
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -166,16 +185,16 @@ class DataStreamWindowJoin(
       def createProcTimeInnerJoinFunction(
           leftDataStream: DataStream[CRow],
           rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
           joinFunctionName: String,
           joinFunctionCode: String,
           leftKeys: Array[Int],
           rightKeys: Array[Int]): DataStream[CRow] = {
     
    -    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    -
    -    val procInnerJoinFunc = new ProcTimeWindowInnerJoin(
    --- End diff --
    
    We should remove the previous implementation if we don't use it anymore.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139400726
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    +
    +    val joinProcessFunc = new RowTimeBoundedStreamInnerJoin(
    +      -10, 20, 0, rT, rT, "TestJoinFunction", funcCode, 0, 0)
    +
    +    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[String](1),
    +        new TupleRowKeySelector[String](1),
    +        BasicTypeInfo.STRING_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // Advance
    +    testHarness.processWatermark1(new Watermark(1))
    +    testHarness.processWatermark2(new Watermark(1))
    +
    +    // Test late data
    --- End diff --
    
    A few more comments like this would be good


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140197925
  
    --- 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)) {
    --- End diff --
    
    I think we should try to produce results which are as complete as possible. In order to avoid the situation that you described in the 2. case, I suggested to include `allowedLateness` into the condition that checks if an element should be stored. So the operator time of the other input needs to be adjusted by that.
    
    I think the storing and joining conditions are independent of each other. So I would do something like:
    
    ```
    if (upperWindowBound > otherOpTime - allowedLateness) { // or >= ?
      // store record
    }
    if (myRecord > myOpTime - allowedLateness) { // or >= ?
      // join record
    }
    ```


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137102330
  
    --- 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) {
    --- End diff --
    
    don't we need to check against the other watermark, i.e., whether the other stream has progressed so much that this record cannot be joined with any record from the other side any more? 
    
    I think it is not important whether the record is late with respect to other records of this stream but rather whether there are (or may be) record to be joined with.
    
    If all records to join with are already evicted and none are expected in the future, we can dismiss this record. Otherwise, we should check the other cache and put it in our cache.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139437709
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    --- End diff --
    
    make class `final`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139722338
  
    --- 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) {
    --- End diff --
    
    Yes, that's what I meant


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140263201
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.util.{ArrayList, 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.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +  //For delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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 rows from the left stream.
    +    */
    +  override def processElement1(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      leftOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      rightOperatorTime,
    +      rightTimerState,
    +      leftCache,
    +      rightCache,
    +      leftRow = true
    +    )
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      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,
    +      rowTime,
    +      ctx,
    +      out,
    +      rightOperatorTime,
    +      oppositeLowerBound,
    +      oppositeUpperBound,
    +      leftOperatorTime,
    +      leftTimerState,
    +      rightCache,
    +      leftCache,
    +      leftRow = false
    +    )
    +  }
    +
    +  /**
    +    * 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,
    +      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)
    --- End diff --
    
    Ah, I misread your proposal then :-) I think starting from a constant value is a good idea. When making it dynamic we have to be careful. Too high granularity will lead to larger lists and higher ser/de overhead.
    
    Btw. while iterating over the list of a rounded key, we can still clean-up the state using the same condition as before and eagerly reduce the state size.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137014165
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    +            rowTimeInnerJoinFunc,
    +            rowTimeInnerJoinFunc.getMaxOutputDelay)
    --- End diff --
    
    rename method to `getMaxWatermarkDelay`?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139436599
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    --- End diff --
    
    make class `final` to support inlining of method.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139407845
  
    --- 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]],
    --- End diff --
    
    change to `Types.LONG.asInstanceOf[TypeInformation[Long]]`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143401295
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    OK, but the current implementation does not prevent overflows.
    Shouldn't the method rather be:
    
    ```
    val expirationTime = operatorTime - relativeSize - allowedLateness - 1
    if (expirationTime >= 0) {
      expirationTime
    } else {
      Long.MaxValue
    }
    ```


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143231150
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -102,5 +117,154 @@ class JoinITCase extends StreamingWithStateTestBase {
         env.execute()
       }
     
    +  /** test rowtime inner join **/
    +  @Test
    +  def testRowTimeInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    +
    +    val sqlQuery =
    +      """
    +        |SELECT t2.a, t2.c, t1.c
    +        |FROM T1 as t1 join T2 as t2 ON
    +        |  t1.a = t2.a AND
    +        |  t1.rt BETWEEN t2.rt - INTERVAL '5' SECOND AND
    +        |    t2.rt + INTERVAL '6' SECOND
    +        |""".stripMargin
    +
    +    val data1 = new mutable.MutableList[(Int, Long, String, Long)]
    --- End diff --
    
    If I understand correctly, this problem was caused by the different semantics of `Null` in SQL and other common languages (i.e., `Null =? Null`). We transform the equi-conditions to a `keyBy` operation and maybe rows with `Null` keys are mapped to the same group, thus being taken as identical?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137205581
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    --- End diff --
    
    Yes. All the methods that need to be implemented differently have been marked. I plan to create one base class and two subclasses for proctime and rowtime, separately.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142762915
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    +      operatorTime - relativeSize - allowedLateness - 1
    +    } else {
    +      Long.MaxValue
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up rows in a specified time.
    +    *
    +    * @param ctx        the context to register timer
    +    * @param rowTime    time for the input row
    +    * @param timerState stores the timestamp for the next timer
    +    * @param leftRow    whether this row comes from the left stream
    +    */
    +  private def registerCleanUpTimer(
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
    +    }
    +    registerTimer(ctx, cleanupTime)
    +    timerState.update(cleanupTime)
    +  }
    +
    +  /**
    +    * Remove the expired rows. Register a new timer if the cache still holds valid rows
    +    * after the cleaning up.
    +    *
    +    * @param expirationTime the expiration time for this cache
    +    * @param rowCache       the row cache
    +    * @param timerState     timer state for the opposite stream
    +    * @param ctx            the context to register the cleanup timer
    +    * @param removeLeft     whether to remove the left rows
    +    */
    +  private def removeExpiredRows(
    +      expirationTime: Long,
    +      rowCache: MapState[Long, JList[Row]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = rowCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var rowTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      rowTime = keysIterator.next
    +      if (rowTime <= expirationTime) {
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (rowTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = rowTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(
    +        ctx,
    +        earliestTimestamp,
    +        timerState,
    +        removeLeft)
    +    } else {
    +      // The timerState will be 0.
    --- End diff --
    
    Add a comment. "No rows left in the cache. Clear all state."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137012982
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    --- End diff --
    
    Rather have two subclasses than dealing with everything in a single class?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137137915
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala ---
    @@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
     
         if (windowBounds.isDefined) {
           if (windowBounds.get.isEventTime) {
    -        // we cannot handle event-time window joins yet
    -        false
    +        val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +          .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
    +
    +        !remainingPredsAccessTime && !procTimeAttrInOutput
    --- End diff --
    
    Shall we also keep the rowtime attributes in the outputs of proctime join?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137112342
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    --- End diff --
    
    ```
    val cleanUpTime = if (leftRecord) {
      timeForRecord + leftRelativeSize + allowedLateness + 1
    } else {
      timeForRecord + rightRelativeSize + allowedLateness + 1
    }
    ```
    would be easier to read, IMO.



---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137108027
  
    --- 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
    --- End diff --
    
    we can define this inside of the loop as `val` (same for `oppositeTime`)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139446305
  
    --- 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)) {
    --- End diff --
    
    I thought about this condition again. I think we need to distinguish two cases:
    
    1. **Storing a record in state**: We need to put a row into state if we must expect qualifying rows on the other side to arrive, i.e., the time of other input is before the upper window boundary of current row (adapted by allowed lateness). Hence, we also need to check the other input's watermark when accessing (inserting) the cache of the current input (same pattern as for cleaning state).
    2. **Joining with other side**: We need to iterate over the other input's cache if there might be qualifying rows in the cache. Since we clean the other cache based on the watermark of the current input, we need to check the rowtime against the watermark of the current input (adjusted by allowed lateness). This is the check that you have right now.
    
    Both might be true, just one, or none depending on the progress of the watermarks and the window boundaries.
    
    So, I would move the code to insert the row into the cache out of this condition and wrap it in another condition that checks against the other input's watermark.
    
    Please check if this make sense. It's quite easy to get confused ;-)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139386347
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -196,53 +215,69 @@ class DataStreamWindowJoin(
         }
       }
     
    +  def createEmptyInnerJoinFunction(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow]) = {
    +    leftDataStream.connect(rightDataStream).process(
    +      new CoProcessFunction[CRow, CRow, CRow] {
    +        override def processElement1(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +        override def processElement2(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +      })
    +  }
       def createRowTimeInnerJoinFunction(
    --- End diff --
    
    Add newline


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142714462
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    --- End diff --
    
    Add comment "Upper bound of current join window has not passed the cache expiration time yet. There might be qualifying rows in the cache that the current row needs to be joined with."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139435309
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
         }
    +    registerTimer(ctx, !leftRow, cleanupTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRow) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRow = ${rowTime}, cleanupTime should be ${cleanupTime - cleanupDelay}," +
    +      s" but delayed to ${cleanupTime}," +
    +      s" oppositeWatermark = ${oppositeWatermark}")
    +    timerState.update(cleanupTime)
    +    //if cleanupTime <= oppositeWatermark + allowedLateness && firstTimer, we may set the
    +    //  backPressureSuggestion =
    +    //    if (leftRow) (oppositeWatermark + allowedLateness - cleanupTime)
    +    //    else -(oppositeWatermark + allowedLateness - cleanupTime)
       }
     
    -
       /**
         * Called when a registered timer is fired.
    -    * Remove records which are earlier than the expiration time,
    -    * and register a new timer for the earliest remaining records.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
         *
         * @param timestamp the timestamp of the timer
         * @param ctx       the context to register timer or get current time
         * @param out       the collector for returning result values
         */
       override def onTimer(
    -    timestamp: Long,
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    -    out: Collector[CRow]): Unit = {
    -    getCurrentOperatorTime(ctx)
    -    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    -    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    -    //TODO already been emitted by the source.
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
         if (leftTimerState.value == timestamp) {
           val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    -      removeExpiredRecords(
    -        timestamp,
    +      removeExpiredRows(
             rightExpirationTime,
             leftOperatorTime,
             rightCache,
             leftTimerState,
             ctx,
    -        false
    +        removeLeft = false
           )
         }
     
         if (rightTimerState.value == timestamp) {
           val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    -      removeExpiredRecords(
    -        timestamp,
    +      removeExpiredRows(
             leftExpirationTime,
             rightOperatorTime,
             leftCache,
             rightTimerState,
             ctx,
    -        true
    +        removeLeft = true
           )
         }
       }
     
       /**
    -    * Remove the expired records. Register a new timer if the cache still holds records
    +    * Remove the expired rows. Register a new timer if the cache still holds valid rows
         * after the cleaning up.
    +    *
    +    * @param expirationTime    the expiration time for this cache
    +    * @param oppositeWatermark the watermark of the opposite stream
    +    * @param rowCache          the row cache
    +    * @param timerState        timer state for the opposite stream
    +    * @param ctx               the context to register the cleanup timer
    +    * @param removeLeft        whether to remove the left rows
         */
    -  private def removeExpiredRecords(
    -    timerFiringTime: Long,
    -    expirationTime: Long,
    -    oppositeWatermark: Long,
    -    recordCache: MapState[Long, JList[Row]],
    -    timerState: ValueState[Long],
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    -    removeLeft: Boolean): Unit = {
    +  private def removeExpiredRows(
    +      expirationTime: Long,
    +      oppositeWatermark: Long,
    +      rowCache: MapState[Long, JList[Row]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      removeLeft: Boolean): Unit = {
     
    -    val keysIterator = recordCache.keys().iterator()
    +    val keysIterator = rowCache.keys().iterator()
     
         // Search for expired timestamps.
         // If we find a non-expired timestamp, remember the timestamp and leave the loop.
         // This way we find all expired timestamps if they are sorted without doing a full pass.
         var earliestTimestamp: Long = -1L
    -    var recordTime: Long = 0L
    +    var rowTime: Long = 0L
         while (keysIterator.hasNext) {
    -      //TODO The "short-circuit" code was commented, because when using a StateMap with
    -      //TODO unordered keys, the cache will grow indefinitely!
    -      // && earliestTimestamp < 0) {
    -      recordTime = keysIterator.next
    -      if (recordTime <= expirationTime) {
    -        // TODO Not sure if we can remove records directly.
    +      rowTime = keysIterator.next
    +      if (rowTime <= expirationTime) {
             keysIterator.remove()
           } else {
             // We find the earliest timestamp that is still valid.
    -        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    -          earliestTimestamp = recordTime
    +        if (rowTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = rowTime
             }
           }
         }
         // If the cache contains non-expired timestamps, register a new timer.
         // Otherwise clear the states.
         if (earliestTimestamp > 0) {
    -      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +      registerCleanUpTimer(
    +        ctx,
    +        earliestTimestamp,
    +        oppositeWatermark,
    +        timerState,
    +        removeLeft,
    +        firstTimer = false)
         } else {
           // The timerState will be 0.
           timerState.clear()
    -      recordCache.clear()
    +      rowCache.clear()
         }
       }
     
       /**
    -    * Get the operator times of the two streams.
    +    * Check if the row is out of date.
    +    *
    +    * @param timeForRow time of the row
    +    * @param watermark  watermark for the stream
    +    * @return true if the row is out of date; false otherwise
    +    */
    +  def checkRowOutOfDate(timeForRow: Long, watermark: Long): Boolean
    --- End diff --
    
    rename method to `isRowTooLate()`?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142891284
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    --- End diff --
    
    Forgive me for the repeated silly sequelae 🤦‍♂️ 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143445293
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    There are two cleanup-related times: (1) the expected cleanup time, which is decided by the rowtime, used for registering the timer and triggered by the operator time; and (2) the real expiration time, which is calculated by the operator time (the code snippet we are discussing). In most cases, (1) the expected cleanup time will not overflow (i.e. the real rowtime will never approach `Long.MaxValue`) and (2) the expiration time calculated in the `onTimer()` method should be greater than or equal to some timestamps of the cached records, otherwise there will be no records to be removed.
    
    There is only one situation we should pay attention to: the expected cleanup time > 0, while the real expiration time < 0 (caused by overflow). In that case, no records will be removed, thus we register a new timer with a positive timestamp. This timer will be triggered by the `Long.MaxValue` operator time, but again no records will be removed and a new timer will be registered. It's a dead loop.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143222605
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +229,50 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
    +      joinFunctionName: String,
    +      joinFunctionCode: String,
    +      leftKeys: Array[Int],
    +      rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val rowTimeInnerJoinFunc = new RowTimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness = 0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx)
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    --- End diff --
    
    Actually, I was quite confused about this condition `!leftKeys.isEmpty`. Since in `FlinkLogicalJoin.scala`, queries without equi-conditions are blocked, when will this condition be hold?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137096753
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    --- End diff --
    
    split this method into two, one for left and one for right records to avoid the `isLeft` check.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137042184
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    --- End diff --
    
    this method should also become an abstract method that is implemented by subclasses. Also the name suggests that it returns the current operator time but is does not return a value. Rename it to `updateOperatorTime()`?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143400792
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    --- End diff --
    
    OK, that makes sense. Thanks for the explanation. Can you add a brief comment to explain the condition? 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139402125
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    --- End diff --
    
    These tests are a good start. 
    We also need a few (2 +/- 1) end-to-end tests (see `org.apache.flink.table.runtime.stream.sql.JoinITCase`) that check the correct translation and execution (incl. result validation).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139723927
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    --- End diff --
    
    we can also leave it here. It doesn't affect the semantics of the join and should have not much effect if you eagerly clean the state in the `processElement()` methods.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137144634
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    --- End diff --
    
    I'd like to call this kind of join "time-bounded join" instead of "window join". When referring to window join, the users may think of tumbling-window or sliding-window, while they are actually not the same. However, as the “window-join” name has been widely used, I can also accept it. Do you have any idea about that?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139591493
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = {
    +    timeForRow <= watermark - allowedLateness
    +  }
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    --- End diff --
    
    Yes, you are right. I'll move this check to places where we actually use the watermark.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137228034
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    --- End diff --
    
    Yes, you are right.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142689196
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +229,50 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
    +      joinFunctionName: String,
    +      joinFunctionCode: String,
    +      leftKeys: Array[Int],
    +      rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val rowTimeInnerJoinFunc = new RowTimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness = 0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx)
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "InnerRowtimeWindowJoin",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    +            rowTimeInnerJoinFunc,
    +            rowTimeInnerJoinFunc.getMaxOutputDelay)
    +        )
    +    } else {
    +      leftDataStream.connect(rightDataStream)
    +        .keyBy(new NullByteKeySelector[CRow](), new NullByteKeySelector[CRow])
    +        .transform(
    +          "InnerRowtimeWindowJoin",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    --- End diff --
    
    `KEY` type is `Byte` instead of `CRow`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137040219
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        return if (isLeft) {
    --- End diff --
    
    No `return` in Scala (unless necessary)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137118783
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    --- End diff --
    
    I think you are right, short-cutting doesn't work. The worst case would be if the keys are in descending timestamp order which might happen if there are no order guarantees.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140266297
  
    --- 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)) {
    --- End diff --
    
    Fine. Let's produce as many results as we can!


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137012563
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    --- End diff --
    
    change to `allowedLateness = 0L,`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137111384
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    --- End diff --
    
    No, but as I said, I think we need to change the condition. I would also not add a log statement here (esp. not on WARN level).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139387007
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -166,16 +185,16 @@ class DataStreamWindowJoin(
       def createProcTimeInnerJoinFunction(
    --- End diff --
    
    rename method to `createProcTimeInnerJoin()` as it does not return the function but a joined stream.



---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142705409
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    --- End diff --
    
    change comment to "Check if we need to join the current row against cached rows of the right input."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137101377
  
    --- 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?
    --- End diff --
    
    this is fine. We want to avoid method calls and conditions as much possible (without causing too much code duplication).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143487841
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    If we still want to check the overflow, the condition need to be more precise.
    ```
    val expirationTime = operatorTime - relativeSize - allowedLateness - 1
    if (expirationTime < 0 && relativeSize + allowedLateness + 1 < 0) {
        Long.MaxValue
    } else {
        expirationTime
    }
    ```
    If we just check the end of the stream and ignore the other overflow circumstances, I think the original code also works.
    ```
    if (operatorTime < Long.MaxValue) {
        operatorTime - relativeSize - allowedLateness - 1
    } else {
        Long.MaxValue
    }
    ```
    Which one do you prefer?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137037638
  
    --- 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
    --- End diff --
    
    please describe the parameters


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137020297
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala ---
    @@ -39,14 +39,17 @@ import scala.collection.JavaConverters._
       */
     object WindowJoinUtil {
     
    -  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, leftUpperBound: Long)
    +  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, leftUpperBound: Long,
    --- End diff --
    
    change formatting to
    ```
    case class WindowBounds(
      isEventTime: Boolean,
      leftLowerBound: Long,
      ...,
      rightTimeIdx: Int)
    ```
    similar to `TimePredicate`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142699858
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    +      operatorTime - relativeSize - allowedLateness - 1
    +    } else {
    +      Long.MaxValue
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up rows in a specified time.
    +    *
    +    * @param ctx        the context to register timer
    +    * @param rowTime    time for the input row
    +    * @param timerState stores the timestamp for the next timer
    +    * @param leftRow    whether this row comes from the left stream
    +    */
    +  private def registerCleanUpTimer(
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      timerState: ValueState[Long],
    --- End diff --
    
    `timerState` can be picked based on `leftRow` flag, no need to have it as an argument.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142691841
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,77 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +final class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx) {
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    leftOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +      else 0L
    +    rightOperatorTime =
    --- End diff --
    
    just use `leftOperatorTime` to avoid the additional method calls and condition?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r141993041
  
    --- 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)) {
    --- End diff --
    
    Well, that makes sense to me. To achieve it, I think we should compare the **real lowest timestamp** in the opposite cache with the `oppositeQualifiedUpperBound`, i.e., 
    ```
    if (upperWindowBound > otherOpTime - allowedLateness) {
      // store record
    }
    if (upperWindowBound > realLowestTimestamp) {
      // join record
    }
    ```
    I'll update the implementation and the tests.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142681890
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -152,19 +176,40 @@ class DataStreamWindowJoin(
         }
       }
     
    -  def createProcTimeInnerJoinFunction(
    +  def createEmptyInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow]) = {
    +    leftDataStream.connect(rightDataStream).process(
    +      new CoProcessFunction[CRow, CRow, CRow] {
    +        override def processElement1(
    --- End diff --
    
    add `Unit` return types for both `processElement` methods.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143401988
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -102,5 +117,154 @@ class JoinITCase extends StreamingWithStateTestBase {
         env.execute()
       }
     
    +  /** test rowtime inner join **/
    +  @Test
    +  def testRowTimeInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    +
    +    val sqlQuery =
    +      """
    +        |SELECT t2.a, t2.c, t1.c
    +        |FROM T1 as t1 join T2 as t2 ON
    +        |  t1.a = t2.a AND
    +        |  t1.rt BETWEEN t2.rt - INTERVAL '5' SECOND AND
    +        |    t2.rt + INTERVAL '6' SECOND
    +        |""".stripMargin
    +
    +    val data1 = new mutable.MutableList[(Int, Long, String, Long)]
    --- End diff --
    
    Yes, you understood the problem correctly. Without PR #4732, join keys are mapped to `Tuple` which do not support null fields unless the field type supports them (which is the case for String but not other primitives). With #4732 keys are mapped to `Row` which supports null fields but treats `null == null`. Therefore, we need to add these predicates into the code-gen'd conditions that correctly evaluate the predicates according to three-value logic.
    
    After #4732 all types will support null value keys.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142710061
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    --- End diff --
    
    Why do we check against the expiration time that was computed in the previous call of `processElement()`? The operator time might have changed in the meantime.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137120634
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    --- End diff --
    
    clean up is quite expensive because we iterate over the full map. By scheduling a timer to the smallest not expired timestamp, we will do this soon again. This might be a problem in case of large window sizes. Does it make sense to delay the clean up by adding a half window size or setting the timer to the average of min and max timestamp?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137158818
  
    --- 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
    --- End diff --
    
    This variable is used to store a suggestion value for performing backpressure *in the future*. We could cache less records if one of the stream is held back with this suggestion. It's just like moving the cache from Flink to upstream components (e.g., Kafka).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137039075
  
    --- 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;
    --- End diff --
    
    add return type


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143357544
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -102,5 +117,154 @@ class JoinITCase extends StreamingWithStateTestBase {
         env.execute()
       }
     
    +  /** test rowtime inner join **/
    +  @Test
    +  def testRowTimeInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    +
    +    val sqlQuery =
    +      """
    +        |SELECT t2.a, t2.c, t1.c
    +        |FROM T1 as t1 join T2 as t2 ON
    +        |  t1.a = t2.a AND
    +        |  t1.rt BETWEEN t2.rt - INTERVAL '5' SECOND AND
    +        |    t2.rt + INTERVAL '6' SECOND
    +        |""".stripMargin
    +
    +    val data1 = new mutable.MutableList[(Int, Long, String, Long)]
    --- End diff --
    
    Currently not all types are supported for `null` keys. I only use the `String` type for test.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139435951
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForLeftStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    --- End diff --
    
    should be `leftOperatorTime` to ensure that both are the same?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137043440
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        return if (isLeft) {
    +          record.row.getField(leftTimeIdx).asInstanceOf[Long]
    +        } else {
    +          record.row.getField(rightTimeIdx).asInstanceOf[Long];
    +        }
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        return context.timerService().currentProcessingTime();
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a proctime or rowtime timer.
    +    *
    +    * @param ctx         the context to register the timer
    +    * @param isLeft      whether this timer should be registered on the left stream
    +    * @param cleanupTime timestamp for the timer
    +    */
    +  protected def registerTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context, isLeft: Boolean, cleanupTime: Long): Unit = {
    +    // Maybe we can register timers for different streams in the future.
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    --- End diff --
    
    case blocks don't need to be in curly braces:
    ```
    case JoinTimeIndicator.ROWTIME =>
      ctx.timerService.registerEventTimeTimer(cleanupTime)
    ```


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139407531
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -55,18 +49,18 @@ import org.apache.flink.util.Collector
       * @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]
    +abstract 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)
    --- End diff --
    
    We can remove `timeIndicator` and `JoinTimeIndicator`. They are only used for the state names which do not need to distinguish row and processing time.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137150128
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala ---
    @@ -115,10 +118,15 @@ object WindowJoinUtil {
           case _ =>
             Some(otherPreds.reduceLeft((l, r) => RelOptUtil.andJoinFilters(rexBuilder, l, r)))
         }
    -
    -    val bounds = Some(WindowBounds(timePreds.head.isEventTime, leftLowerBound, leftUpperBound))
    -
    -    (bounds, remainCondition)
    +    if (timePreds.head.leftInputOnLeftSide) {
    --- End diff --
    
    We should also subtract the `leftLogicalFieldCnt` for the later index?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139424544
  
    --- 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) {
    --- End diff --
    
    we can check here if `oppositeTime` < `myWatermark` +/- (windowsize + allowedLateness) and discard the state in this case.
    
    This would be the eager cleanup optimization. This is not only desirable because it reduces the state size but also improves join performance because we don't have to iterate over expired state entries.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139386887
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -196,53 +215,69 @@ class DataStreamWindowJoin(
         }
       }
     
    +  def createEmptyInnerJoinFunction(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow]) = {
    +    leftDataStream.connect(rightDataStream).process(
    +      new CoProcessFunction[CRow, CRow, CRow] {
    +        override def processElement1(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +        override def processElement2(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +      })
    +  }
       def createRowTimeInnerJoinFunction(
    --- End diff --
    
    rename method to `createRowTimeInnerJoin()` as it does not return the function but a joined stream.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142762761
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    +      operatorTime - relativeSize - allowedLateness - 1
    +    } else {
    +      Long.MaxValue
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up rows in a specified time.
    +    *
    +    * @param ctx        the context to register timer
    +    * @param rowTime    time for the input row
    +    * @param timerState stores the timestamp for the next timer
    +    * @param leftRow    whether this row comes from the left stream
    +    */
    +  private def registerCleanUpTimer(
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
    +    }
    +    registerTimer(ctx, cleanupTime)
    +    timerState.update(cleanupTime)
    +  }
    +
    +  /**
    +    * Remove the expired rows. Register a new timer if the cache still holds valid rows
    +    * after the cleaning up.
    +    *
    +    * @param expirationTime the expiration time for this cache
    +    * @param rowCache       the row cache
    +    * @param timerState     timer state for the opposite stream
    +    * @param ctx            the context to register the cleanup timer
    +    * @param removeLeft     whether to remove the left rows
    +    */
    +  private def removeExpiredRows(
    +      expirationTime: Long,
    +      rowCache: MapState[Long, JList[Row]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = rowCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var rowTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      rowTime = keysIterator.next
    +      if (rowTime <= expirationTime) {
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (rowTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = rowTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(
    --- End diff --
    
    Add a comment. "There are rows left in the cache. Register a timer to expire them later."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r140251765
  
    --- 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)) {
    --- End diff --
    
    Yes, we must include the `allowedLateness` into all the conditions. However, I think the "storing condition" should be covered by the "lateness condition", i.e.,
    ```
    if (myRecord > myOpTime - allowedLateness) {
        if (upperWindowBound > otherOpTime - allowedLateness) {
            // store record
        }
      // join record
    }
    ```
    That's because once we store a record `R` which doesn't pass the lateness check, it may be joined with a later coming record form the opposite stream. Now that `R` isn't joined when it comes, to keep the completeness, it should never be joined.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137038607
  
    --- 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
    --- End diff --
    
    What is the `backPressureSuggestion` for?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137040696
  
    --- 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)
    --- End diff --
    
    Please name `boolean` parameters: `true` -> `isLeft = true`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137229356
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    --- End diff --
    
    Right, I think for now, I'd add a window size to the `earliestTimestamp` for the clean up.
    In addition, we can add a check to the cache iteration in `processElement()` that compares the cached records against a cleanup threshold (which is based on the current watermark, window size, and allowedLateness). Then we can also cleanup during regular processing and the cleanup timer would be a safety net.
    
    Since we need to go over the full cache for joining anyway, we can also cleanup at that point.


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Thanks for the review, @fhueske. This PR is a little rough when I committed. I'll address your comments and submit a refined version as soon as possible.
    
    Best, Xingcan


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137043506
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Return the rowtime or proctime for the target record.
    +    *
    +    * @param context the runtime context
    +    * @param record  the target record
    +    * @param isLeft  whether the record is from the left stream
    +    * @return time for the target record
    +    */
    +  protected def getTimeForRecord(
    +    context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    record: CRow,
    +    isLeft: Boolean): Long = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        return if (isLeft) {
    +          record.row.getField(leftTimeIdx).asInstanceOf[Long]
    +        } else {
    +          record.row.getField(rightTimeIdx).asInstanceOf[Long];
    +        }
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        return context.timerService().currentProcessingTime();
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a proctime or rowtime timer.
    +    *
    +    * @param ctx         the context to register the timer
    +    * @param isLeft      whether this timer should be registered on the left stream
    +    * @param cleanupTime timestamp for the timer
    +    */
    +  protected def registerTimer(
    --- End diff --
    
    Make an abstract method


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142682283
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -152,19 +176,40 @@ class DataStreamWindowJoin(
         }
       }
     
    -  def createProcTimeInnerJoinFunction(
    +  def createEmptyInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow]) = {
    +    leftDataStream.connect(rightDataStream).process(
    +      new CoProcessFunction[CRow, CRow, CRow] {
    +        override def processElement1(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +        override def processElement2(
    +          value: CRow,
    +          ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +          out: Collector[CRow]) = {
    +          //Do nothing.
    +        }
    +      })
    --- End diff --
    
    add a `returns(returnTypeInfo)` call to ensure we use the right type.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139983961
  
    --- 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)) {
    --- End diff --
    
    Hi @fhueske, it takes me a little time to rethink these conditions. As you said, it's really quite easy to get confused 😄 
    
    In general, I'm in favour of your suggestions. Besides, I've got some extra ideas. 
    1. About storing a record in state. Only if `oppositeOperatorTime < oppositeQualifedUpperBound`, can a record be stored. For example, suppose there's a record from the left stream with `time = 10` and qualified time range of the right stream is calculated to be `[8, 15]`. Then only if the `rightOperatorTime < 15`, need we cache the left record. Otherwise, it means all the qualified records from the right stream should have been seen.
    2. We need two conditions here, but the "lateness condition" should be applied out of the "storing condition", i.e., we first check the lateness of a record and only if it passes can we continue processing it (checking if it should be stored, etc.). That's because even if there exist records from the opposite stream that can be joined with a late record, we can not guarantee the **completeness** of the results. In the above example, if `rightOperatorTime = 16`, the left record with `time = 10` will not be stored. After that, here comes a right record with `time = 15`, but we can never join it with the left record just seen before since it is not stored.
    
    "Incomplete" or "empty", which one to choose?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137119180
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    --- End diff --
    
    I think it can be removed.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137227189
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    +            rowTimeInnerJoinFunc,
    +            rowTimeInnerJoinFunc.getMaxOutputDelay)
    --- End diff --
    
    OK, let's keep `getMaxOutputDelay()` but we should improve the documentation of the method and make clear that this returns the maximum interval between receiving a row and emitting it (as part of a joined row).


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139585456
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForLeftStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    --- End diff --
    
    Yes, you are right. To keep them identical, we should return the `leftOperatorTime` here. However, this makes `updateOperatorTime` and `getTimeForLeftStream` coupled, i.e., `updateOperatorTime` must be invoked before `getTimeForLeftStream`. Can we bear this? 
    
    I've got an idea about the processing time. How about temporarily caching the machine time for the same `StreamRecord` instead of invoking the `System.currentTimeMillis()` each time?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143417404
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    But what does the current check prevent then? We might still run in an overflow if `relativeSize` is negative. Moreover, the case of `operatorTime == Long.MaxValue` may only happen at the end of the job when no more records will be received. In this case the overflow would have no consequence, right? Doesn't it make more sense to prevent the registration of negative timers?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137099832
  
    --- 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) {
    --- End diff --
    
    can this check be removed? It seems that nothing will happen if `relativeWindowSize <=0`, so we could also reject this case in the constructor, right?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137272749
  
    --- 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) {
    --- End diff --
    
    That's a good idea. I'll move this check out of the function.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143448956
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    I'll change the condition here to be more precise.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137117454
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    --- End diff --
    
    `timerFiringTime` is not used can can be removed.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137201317
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    --- End diff --
    
    That's indeed a critical problem. The current approach is designed for minimizing the cached records, but may spend too much time in cleaning up. I think of two ways to improve that: (1) as you suggested, we delay the timers with specific spans, or (2) we schedule the cleaning up process with a specific interval according to the relative window size. What do you think?
    Actually, every problem will be solved if we got a state with ordered keys, like the TreeMap ;-)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139386829
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -196,53 +215,69 @@ class DataStreamWindowJoin(
         }
       }
     
    +  def createEmptyInnerJoinFunction(
    --- End diff --
    
    rename method to `createEmptyInnerJoin()` as it does not return the function but a joined stream.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139439684
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = {
    +    timeForRow <= watermark - allowedLateness
    +  }
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    --- End diff --
    
    We need this check to avoid an underflow in case the current watermark is `Long.MIN_VALUE` correct?
    I see that it is nice to encapsulate the check here, but would we need fewer checks if we would perform this check every time we actually use the watermark?


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Hi @fhueske, the PR has been updated. Temporarily, I keep the logic for dealing with the late data, as well as the fine-grained cache. 
    
    For the late data semantics problem, I think we need to rethink it and make a final decision (maybe we should consult others). For the cache optimization problem, I want to leave it a future work. 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137014641
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +195,54 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoinFunction(
    +    leftDataStream: DataStream[CRow],
    +    rightDataStream: DataStream[CRow],
    +    joinFunctionName: String,
    +    joinFunctionCode: String,
    +    leftKeys: Array[Int],
    +    rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    +    val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME
    +    )
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "rowTimeInnerJoinFunc",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    +            rowTimeInnerJoinFunc,
    +            rowTimeInnerJoinFunc.getMaxOutputDelay)
    +        ).returns(returnTypeInfo)
    --- End diff --
    
    The `returns()` call might be redundant as the `returnTypeInfo` is already set in the operator.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137098896
  
    --- 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
    --- End diff --
    
    can be computed in `processElement1()` and `processElement2()` without checking the `leftRecord` condition.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139631978
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = {
    +    timeForRow <= watermark - allowedLateness
    +  }
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    --- End diff --
    
    After checking the codes, I find out that the number of watermark usages is even larger than that of their updates (i.e., each update is followed by at least one usage). Considering that the number of usage may increase in the future, I suggest to keep this check in the update method. Of course, it would be better if we can use a "safer" initial value for
     the watermark😄. What do you think? 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139427057
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    +    } else {
    +      rowTime + rightRelativeSize + cleanupDelay + allowedLateness + 1
         }
    +    registerTimer(ctx, !leftRow, cleanupTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRow) "RIGHT" else "LEFT"} state:"
    --- End diff --
    
    Please remove debug log


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143412087
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    Thanks for pointing this out, @fhueske. Actually the `expirationTime` can be negative when the job has just been started (e.g., when operatorTime = 10 and relativeSize = 20). I'd rather take the Long.MaxValue for `operatorTime` as a special case and keep the original logic. What do you think?


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Hi @xccui, I think aside from the equi join predicate limitation, all issues have been addressed. This can be fixed in a follow up task, IMO.
    I'll have a last look at the PR and hopefully merge it.
    
    Thanks for the great work,
    Fabian
    
    Follow ups will be (some might already have JIRAs):
    - enabling streaming joins for the Table API
    - blocking of state entries and clean-up improvements
    - support for windowed non-equi joins
    - support for windowed outer joins


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137093911
  
    --- 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(
    --- End diff --
    
    Add logging about the code compilation as in the other operators


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139628583
  
    --- 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.
       }
     
       /**
    -    * Register a timer for cleaning up records in a specified time.
    +    * Register a timer for cleaning up rows in a specified time.
         *
         * @param ctx               the context to register timer
    -    * @param timeForRecord     time for the input record
    +    * @param rowTime           time for the input row
         * @param oppositeWatermark watermark of the opposite stream
         * @param timerState        stores the timestamp for the next timer
    -    * @param leftRecord        record from the left or the right stream
    +    * @param leftRow           whether this row comes from the left stream
         * @param firstTimer        whether this is the first timer
         */
       private def registerCleanUpTimer(
    -    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    -    timeForRecord: Long,
    -    oppositeWatermark: Long,
    -    timerState: ValueState[Long],
    -    leftRecord: Boolean,
    -    firstTimer: Boolean): Unit = {
    -    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    -      allowedLateness + 1
    -    registerTimer(ctx, !leftRecord, cleanUpTime)
    -    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    -      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    -      s"${oppositeWatermark}")
    -    timerState.update(cleanUpTime)
    -    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    -      backPressureSuggestion =
    -        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    -        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    -      LOG.warn("The clean timer for the " +
    -        s"${if (leftRecord) "left" else "right"}" +
    -        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    -        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    -        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      rowTime: Long,
    +      oppositeWatermark: Long,
    +      timerState: ValueState[Long],
    +      leftRow: Boolean,
    +      firstTimer: Boolean): Unit = {
    +    val cleanupTime = if (leftRow) {
    +      rowTime + leftRelativeSize + cleanupDelay + allowedLateness + 1
    --- End diff --
    
    Thanks for this suggestion and I understand your means here. 
    
    IMO, as long as the stream is continuous, the timer should always be registered in the `removeExpiredRows()`, except for the first one. Thus in real applications, I think there's not so much difference whether we apply the `cleanupDelay` in `removeExpiedRows()` or in `registerCleanUpTimer()`. Moreover, calculating cleanup time differently may confuse the developers/users and it will also make the test complicated (i.e., not easy to decide when a row should be removed). What do you think?


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142697408
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    --- End diff --
    
    Add comment for the branch of the condition like "Operator time of right stream has not exceeded the upper window bound of the current record. We must expect records from the right stream that join with the current record."


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142769397
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -102,5 +117,154 @@ class JoinITCase extends StreamingWithStateTestBase {
         env.execute()
       }
     
    +  /** test rowtime inner join **/
    +  @Test
    +  def testRowTimeInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    +
    +    val sqlQuery =
    +      """
    +        |SELECT t2.a, t2.c, t1.c
    +        |FROM T1 as t1 join T2 as t2 ON
    +        |  t1.a = t2.a AND
    +        |  t1.rt BETWEEN t2.rt - INTERVAL '5' SECOND AND
    +        |    t2.rt + INTERVAL '6' SECOND
    +        |""".stripMargin
    +
    +    val data1 = new mutable.MutableList[(Int, Long, String, Long)]
    --- End diff --
    
    Add two rows with null keys on both sides within join window boundaries to test that join predicates on null values are not evaluated to true. For this to work we need to also fix the `keyBy()` calls to support partitioning of null keys (see #4732)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137113304
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    --- End diff --
    
    its not a problem if the clean up timer is lower than the watermark. In that case, the timer fires immediately.
    
    I'm not sure about the purpose of the backpressure suggestion. AFAIK, there is no mechanism to control the backpressure in Flink. It's rather a self-regulating mechanism.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139387143
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -196,53 +215,69 @@ class DataStreamWindowJoin(
         }
       }
     
    +  def createEmptyInnerJoinFunction(
    --- End diff --
    
    Move method above `createProcTimeInnerJoinFunction()`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137038433
  
    --- 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;
    --- End diff --
    
    remove `;`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142694926
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    --- End diff --
    
    Move this method to `RowTimeBoundedStreamInnerJoin`


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4625


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137100691
  
    --- 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]()
    --- End diff --
    
    Initialize `ArrayList` with initial size 1, we don't need more capacity.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139986193
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -383,13 +384,158 @@ class JoinHarnessTest extends HarnessTestBase{
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(2: JInt, "aaa2", 2: JInt, "bbb7"), true), 7))
    +      CRow(Row.of(2L: JLong, "aaa2", 2L: JLong, "bbb7"), true), 7))
         expectedOutput.add(new StreamRecord(
    -      CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12))
    +      CRow(Row.of(1L: JLong, "aaa3", 1L: JLong, "bbb12"), true), 12))
     
         verify(expectedOutput, result, new RowResultSortComparator())
     
         testHarness.close()
       }
     
    +  /** a.c1 >= b.rowtime - 10 and a.rowtime <= b.rowtime + 20 **/
    +  @Test
    +  def testCommonRowTimeJoin() {
    --- End diff --
    
    Oh, sorry I miss this part. Will add soon.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137227455
  
    --- 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
    --- End diff --
    
    AFAIK, there are no plans for a feature like that. This would also need to be integrated with the DataStream API and all connectors which means it is a major change and requires careful design.
    
    I would remove the code for now.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142688996
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +229,50 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
    +      joinFunctionName: String,
    +      joinFunctionCode: String,
    +      leftKeys: Array[Int],
    +      rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val rowTimeInnerJoinFunc = new RowTimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness = 0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx)
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    +        .transform(
    +          "InnerRowtimeWindowJoin",
    +          returnTypeInfo,
    +          new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, CRow](
    --- End diff --
    
    In the current implementation the `KEY` type would be a `Tuple`, but I think we can just pass `_` here. When we adopt #4732, the key will be `Row`.



---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Btw, I noticed I did not reply to your comments. 
    
    I think it would be good to have the eager state cleaning in the initial version. Shouldn't be too much effort. Basically, getting the condition right and calling `remove()` on the `Map.Entry`.
    
    What do you mean by "distinguish the < and <=signs"? If there is an off-by-one issue in the computation of the window boundaries, it needs to be fixed with this PR. We shouldn't merge a semantically incorrect operator (of course there might be bugs...). Performance issues are OK but the semantics must be correct.
    
    Regarding the `"misc"` test failures, yes that can happen. No need to worry about that as long as the `""` libraries build passes. I'll run the tests anyway again before merging ;-)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142685896
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -184,4 +229,50 @@ class DataStreamWindowJoin(
             .returns(returnTypeInfo)
         }
       }
    +
    +  def createRowTimeInnerJoin(
    +      leftDataStream: DataStream[CRow],
    +      rightDataStream: DataStream[CRow],
    +      returnTypeInfo: TypeInformation[CRow],
    +      joinFunctionName: String,
    +      joinFunctionCode: String,
    +      leftKeys: Array[Int],
    +      rightKeys: Array[Int]): DataStream[CRow] = {
    +
    +    val rowTimeInnerJoinFunc = new RowTimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness = 0L,
    +      leftSchema.typeInfo,
    +      rightSchema.typeInfo,
    +      joinFunctionName,
    +      joinFunctionCode,
    +      leftTimeIdx,
    +      rightTimeIdx)
    +
    +    if (!leftKeys.isEmpty) {
    +      leftDataStream
    +        .connect(rightDataStream)
    +        .keyBy(leftKeys, rightKeys)
    --- End diff --
    
    we need to make sure to include the fixes of #4732


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139385974
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala ---
    @@ -117,6 +123,11 @@ class DataStreamWindowJoin(
         val leftKeys = joinInfo.leftKeys.toIntArray
         val rightKeys = joinInfo.rightKeys.toIntArray
     
    +    val relativeWindowSize = leftUpperBound - leftLowerBound
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
    +
    --- End diff --
    
    rm newline


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137110049
  
    --- 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 --
    
    No, we cannot remove records here. We can only remove records when a watermark makes progress. In case of processing time, this is different because record time is always increasing. For event time I would remove this case.


---

[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on the issue:

    https://github.com/apache/flink/pull/4625
  
    Hi @fhueske, I really appreciate for your guidance with great care. Hopefully this prolonged work do not affect the schedule. 
    
    I'll keep working on the follow-up issues.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r142692599
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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,
    --- End diff --
    
    The time indicies are only needed by `RowTimeBoundedStreamInnerJoin`. They can be removed here.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r143211000
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,410 @@
    +/*
    + * 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.util.{ArrayList, List => JList}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.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.api.Types
    +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.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * A CoProcessFunction to execute time-bounded stream inner-join.
    +  * Two kinds of time criteria:
    +  * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X".
    +  *
    +  * @param leftLowerBound  the lower bound for the left stream (X in the criteria)
    +  * @param leftUpperBound  the upper bound for the left stream (Y in the criteria)
    +  * @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
    +  *
    +  */
    +abstract 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)
    +    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 rows from the left stream
    +  private var leftCache: MapState[Long, JList[Row]] = _
    +  // cache to store rows from the right stream
    +  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 var leftExpirationTime: Long = 0L;
    +  private var rightExpirationTime: Long = 0L;
    +
    +  protected var leftOperatorTime: Long = 0L
    +  protected var rightOperatorTime: Long = 0L
    +
    +
    +  // for delayed cleanup
    +  private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2
    +
    +  if (allowedLateness < 0) {
    +    throw new IllegalArgumentException("The allowed lateness must be non-negative.")
    +  }
    +
    +  /**
    +    * Get the maximum interval between receiving a row and emitting it (as part of a joined result).
    +    * Only reasonable for row time join.
    +    *
    +    * @return the maximum delay for the outputs
    +    */
    +  def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +      s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    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]](
    +        "InnerJoinLeftCache",
    +        Types.LONG.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]](
    +        "InnerJoinRightCache",
    +        Types.LONG.asInstanceOf[TypeInformation[Long]],
    +        rightListTypeInfo)
    +    rightCache = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // Initialize the timer states.
    +    val leftTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long])
    +    leftTimerState = getRuntimeContext.getState(leftTimerStateDesc)
    +
    +    val rightTimerStateDesc: ValueStateDescriptor[Long] =
    +      new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long])
    +    rightTimerState = getRuntimeContext.getState(rightTimerStateDesc)
    +  }
    +
    +  /**
    +    * Process rows from the left stream.
    +    */
    +  override def processElement1(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val leftRow = cRowValue.row
    +    val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow)
    +    val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize
    +    val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (rightOperatorTime < rightQualifiedUpperBound) {
    +      // Put the leftRow into the cache for later use.
    +      var leftRowList = leftCache.get(timeForLeftRow)
    +      if (null == leftRowList) {
    +        leftRowList = new ArrayList[Row](1)
    +      }
    +      leftRowList.add(leftRow)
    +      leftCache.put(timeForLeftRow, leftRowList)
    +      if (rightTimerState.value == 0) {
    +        // Register a timer on the RIGHT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForLeftRow, rightTimerState, leftRow = true)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (rightExpirationTime < rightQualifiedUpperBound) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      // Join the leftRow with rows from the right cache.
    +      val rightIterator = rightCache.iterator()
    +      while (rightIterator.hasNext) {
    +        val rightEntry = rightIterator.next
    +        val rightTime = rightEntry.getKey
    +        if (rightTime >= rightQualifiedLowerBound && rightTime <= rightQualifiedUpperBound) {
    +          val rightRows = rightEntry.getValue
    +          var i = 0
    +          while (i < rightRows.size) {
    +            joinFunction.join(leftRow, rightRows.get(i), cRowWrapper)
    +            i += 1
    +          }
    +        }
    +
    +        if (rightTime <= rightExpirationTime) {
    +          // eager remove
    +          rightIterator.remove()
    +        }// We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Process rows from the right stream.
    +    */
    +  override def processElement2(
    +      cRowValue: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    val rightRow = cRowValue.row
    +    val timeForRightRow: Long = getTimeForRightStream(ctx, rightRow)
    +    val leftQualifiedLowerBound: Long = timeForRightRow - leftRelativeSize
    +    val leftQualifiedUpperBound: Long =  timeForRightRow + rightRelativeSize
    +    cRowWrapper.out = out
    +
    +    if (leftOperatorTime < leftQualifiedUpperBound) {
    +      // Put the rightRow into the cache for later use.
    +      var rightRowList = rightCache.get(timeForRightRow)
    +      if (null == rightRowList) {
    +        rightRowList = new ArrayList[Row](1)
    +      }
    +      rightRowList.add(rightRow)
    +      rightCache.put(timeForRightRow, rightRowList)
    +      if (leftTimerState.value == 0) {
    +        // Register a timer on the LEFT stream to remove rows.
    +        registerCleanUpTimer(ctx, timeForRightRow, leftTimerState, leftRow = false)
    +      }
    +    }
    +    // We'd like to produce as many results as possible.
    +    if (leftExpirationTime < leftQualifiedUpperBound) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      // Join the rightRow with rows from the left cache.
    +      val leftIterator = leftCache.iterator()
    +      while (leftIterator.hasNext) {
    +        val leftEntry = leftIterator.next
    +        val leftTime = leftEntry.getKey
    +        if (leftTime >= leftQualifiedLowerBound && leftTime <= leftQualifiedUpperBound) {
    +          val leftRows = leftEntry.getValue
    +          var i = 0
    +          while (i < leftRows.size) {
    +            joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
    +            i += 1
    +          }
    +        }
    +        if (leftTime <= leftExpirationTime) {
    +          // eager remove
    +          leftIterator.remove()
    +        } // We could do the short-cutting optimization here once we get a state with ordered keys.
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove rows whose timestamps are earlier than the expiration time,
    +    * and register a new timer for the remaining rows.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +    updateOperatorTime(ctx)
    +    // In the future, we should separate the left and right watermarks. Otherwise, the
    +    // registered timer of the faster stream will be delayed, even if the watermarks have
    +    // already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      rightExpirationTime = calExpirationTime(leftOperatorTime, rightRelativeSize)
    +      removeExpiredRows(
    +        rightExpirationTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        removeLeft = false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      leftExpirationTime = calExpirationTime(rightOperatorTime, leftRelativeSize)
    +      removeExpiredRows(
    +        leftExpirationTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        removeLeft = true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Calculate the expiration time with the given operator time and relative window size.
    +    *
    +    * @param operatorTime the operator time
    +    * @param relativeSize the relative window size
    +    * @return the expiration time for cached rows
    +    */
    +  private def calExpirationTime(operatorTime: Long, relativeSize: Long): Long = {
    +    if (operatorTime < Long.MaxValue) {
    --- End diff --
    
    Since `relativeSize` may be negative, the expression `operatorTime - relativeSize - allowedLateness - 1` will overflow when `operatorTime = Long.MaxValue` and `relativeSize < 0`. When encountered this, there will be endless timers with negative expiration times (which should have been `Long.MaxValue`) being recursively registered and the job will never stop.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139728162
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/RowTimeBoundedStreamInnerJoin.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * 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 org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute row(event) time bounded stream inner-join.
    +  */
    +class RowTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    leftTimeIdx: Int,
    +    rightTimeIdx: Int)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx,
    +      rightTimeIdx,
    +      JoinTimeIndicator.ROWTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = {
    +    timeForRow <= watermark - allowedLateness
    +  }
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    rightOperatorTime =
    +      if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    --- End diff --
    
    OK, let's keep it here. Changing the value of watermarks won't be possible as it is built into the DataStream API and some users rely on the current behavior. The curse of public APIs ;-)


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137110758
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } else if (oppositeTime > oppositeUpperBound) {
    +            //TODO If the keys are ordered, can we break here?
    --- End diff --
    
    Yes, we could but there is no state backend that guarantees ordered keys. So we have to continue. Please remove the case. You might add a comment that this can be solved more efficiently once we have an ordered state. There is also a JIRA issue that you could link in the comment.


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137040783
  
    --- 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
    --- End diff --
    
    name `boolean` parameter


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137041142
  
    --- 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;
    --- End diff --
    
    please check for semicolons and remove them


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r139423665
  
    --- 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 --
    
    I think we don't need special logic for handling late records. If `allowedLateness > 0`, we simply keep records longer in the state and discard them later. Late records (those that usually would have a ts that's smaller than the other inputs watermark +/- window size) would be handled and joined just as any other record.
    
    Aggregations are a bit different, because we need to update a previously emitted result in case of late data. 


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137092951
  
    --- 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.
    +          } 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
    +              }
    +            }
    +          } 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.")
    +      }
    +    }
    +  }
    +
    +  /**
    +    * Register a timer for cleaning up records in a specified time.
    +    *
    +    * @param ctx               the context to register timer
    +    * @param timeForRecord     time for the input record
    +    * @param oppositeWatermark watermark of the opposite stream
    +    * @param timerState        stores the timestamp for the next timer
    +    * @param leftRecord        record from the left or the right stream
    +    * @param firstTimer        whether this is the first timer
    +    */
    +  private def registerCleanUpTimer(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +    timeForRecord: Long,
    +    oppositeWatermark: Long,
    +    timerState: ValueState[Long],
    +    leftRecord: Boolean,
    +    firstTimer: Boolean): Unit = {
    +    val cleanUpTime = timeForRecord + (if (leftRecord) leftRelativeSize else rightRelativeSize) +
    +      allowedLateness + 1
    +    registerTimer(ctx, !leftRecord, cleanUpTime)
    +    LOG.debug(s"Register a clean up timer on the ${if (leftRecord) "RIGHT" else "LEFT"} state:"
    +      + s" timeForRecord = ${timeForRecord}, cleanUpTime = ${cleanUpTime}, oppositeWatermark = " +
    +      s"${oppositeWatermark}")
    +    timerState.update(cleanUpTime)
    +    if (cleanUpTime <= oppositeWatermark + allowedLateness && firstTimer) {
    +      backPressureSuggestion =
    +        if (leftRecord) (oppositeWatermark + allowedLateness - cleanUpTime)
    +        else -(oppositeWatermark + allowedLateness - cleanUpTime)
    +      LOG.warn("The clean timer for the " +
    +        s"${if (leftRecord) "left" else "right"}" +
    +        s" stream is lower than ${if (leftRecord) "right" else "left"} watermark." +
    +        s" requiredTime = ${formatTime(cleanUpTime)}, watermark = ${formatTime(oppositeWatermark)},"
    +        + s"backPressureSuggestion = " + s"${backPressureSuggestion}.")
    +    }
    +  }
    +
    +
    +  /**
    +    * Called when a registered timer is fired.
    +    * Remove records which are earlier than the expiration time,
    +    * and register a new timer for the earliest remaining records.
    +    *
    +    * @param timestamp the timestamp of the timer
    +    * @param ctx       the context to register timer or get current time
    +    * @param out       the collector for returning result values
    +    */
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    getCurrentOperatorTime(ctx)
    +    //TODO In the future, we should separate the left and right watermarks. Otherwise, the
    +    //TODO registered timer of the faster stream will be delayed, even if the watermarks have
    +    //TODO already been emitted by the source.
    +    if (leftTimerState.value == timestamp) {
    +      val rightExpirationTime = leftOperatorTime - rightRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        rightExpirationTime,
    +        leftOperatorTime,
    +        rightCache,
    +        leftTimerState,
    +        ctx,
    +        false
    +      )
    +    }
    +
    +    if (rightTimerState.value == timestamp) {
    +      val leftExpirationTime = rightOperatorTime - leftRelativeSize - allowedLateness - 1
    +      removeExpiredRecords(
    +        timestamp,
    +        leftExpirationTime,
    +        rightOperatorTime,
    +        leftCache,
    +        rightTimerState,
    +        ctx,
    +        true
    +      )
    +    }
    +  }
    +
    +  /**
    +    * Remove the expired records. Register a new timer if the cache still holds records
    +    * after the cleaning up.
    +    */
    +  private def removeExpiredRecords(
    +    timerFiringTime: Long,
    +    expirationTime: Long,
    +    oppositeWatermark: Long,
    +    recordCache: MapState[Long, JList[Row]],
    +    timerState: ValueState[Long],
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +    removeLeft: Boolean): Unit = {
    +
    +    val keysIterator = recordCache.keys().iterator()
    +
    +    // Search for expired timestamps.
    +    // If we find a non-expired timestamp, remember the timestamp and leave the loop.
    +    // This way we find all expired timestamps if they are sorted without doing a full pass.
    +    var earliestTimestamp: Long = -1L
    +    var recordTime: Long = 0L
    +    while (keysIterator.hasNext) {
    +      //TODO The "short-circuit" code was commented, because when using a StateMap with
    +      //TODO unordered keys, the cache will grow indefinitely!
    +      // && earliestTimestamp < 0) {
    +      recordTime = keysIterator.next
    +      if (recordTime <= expirationTime) {
    +        // TODO Not sure if we can remove records directly.
    +        keysIterator.remove()
    +      } else {
    +        // We find the earliest timestamp that is still valid.
    +        if (recordTime < earliestTimestamp || earliestTimestamp < 0) {
    +          earliestTimestamp = recordTime
    +        }
    +      }
    +    }
    +    // If the cache contains non-expired timestamps, register a new timer.
    +    // Otherwise clear the states.
    +    if (earliestTimestamp > 0) {
    +      registerCleanUpTimer(ctx, earliestTimestamp, oppositeWatermark, timerState, removeLeft, false)
    +    } else {
    +      // The timerState will be 0.
    +      timerState.clear()
    +      recordCache.clear()
    +    }
    +  }
    +
    +  /**
    +    * Get the operator times of the two streams.
    +    *
    +    * @param ctx the context to acquire watermarks
    +    */
    +  protected def getCurrentOperatorTime(
    +    ctx: CoProcessFunction[CRow, CRow, CRow]#Context): Unit = {
    +    timeIndicator match {
    +      case JoinTimeIndicator.ROWTIME => {
    +        rightOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +        leftOperatorTime =
    +          if (ctx.timerService().currentWatermark() > 0) ctx.timerService().currentWatermark()
    +          else 0L;
    +      }
    +      case JoinTimeIndicator.PROCTIME => {
    +        rightOperatorTime = ctx.timerService().currentProcessingTime()
    +        leftOperatorTime = ctx.timerService().currentProcessingTime()
    +      }
    +    }
    +  }
    +
    --- End diff --
    
    remove double newlines


---

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4625#discussion_r137039045
  
    --- 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
    --- End diff --
    
    add return type


---