You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2019/01/11 12:41:17 UTC

[GitHub] kl0u closed pull request #6871: [FLINK-10583][FLINK-10584][table] Add State Retention to temporal joins.

kl0u closed pull request #6871: [FLINK-10583][FLINK-10584][table] Add State Retention to temporal joins.
URL: https://github.com/apache/flink/pull/6871
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/BaseTwoInputStreamOperatorWithStateRetention.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/BaseTwoInputStreamOperatorWithStateRetention.scala
new file mode 100644
index 00000000000..f44d7997c8f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/BaseTwoInputStreamOperatorWithStateRetention.scala
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.join
+
+import java.lang.{Long => JLong}
+import java.util.Optional
+
+import org.apache.flink.annotation.Internal
+import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
+import org.apache.flink.runtime.state.{VoidNamespace, VoidNamespaceSerializer}
+import org.apache.flink.streaming.api.SimpleTimerService
+import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, InternalTimer, Triggerable, TwoInputStreamOperator}
+import org.apache.flink.table.api.{StreamQueryConfig, Types}
+import org.apache.flink.table.runtime.types.CRow
+
+/**
+  * An abstract [[TwoInputStreamOperator]] that allows its subclasses to clean
+  * up their state based on a TTL. This TTL should be specified in the provided
+  * [[StreamQueryConfig]].
+  *
+  * For each known key, this operator registers a timer (in processing time) to
+  * fire after the TTL expires. When the timer fires, the subclass can decide which
+  * state to cleanup and what further action to take.
+  *
+  * This class takes care of maintaining at most one timer per key.
+  *
+  * <p><b>IMPORTANT NOTE TO USERS:</b> When extending this class, do not use processing time
+  * timers in your business logic. The reason is that:
+  *
+  * 1) if your timers collide with clean up timers and you delete them, then state
+  * clean-up will not be performed, and
+  *
+  * 2) (this one is the reason why this class does not allow to override the onProcessingTime())
+  * the onProcessingTime with your logic would be also executed on each clean up timer.
+  */
+@Internal
+abstract class BaseTwoInputStreamOperatorWithStateRetention(
+    queryConfig: StreamQueryConfig)
+  extends AbstractStreamOperator[CRow]
+  with TwoInputStreamOperator[CRow, CRow, CRow]
+  with Triggerable[Any, VoidNamespace] {
+
+  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
+  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
+
+  private val CLEANUP_TIMESTAMP = "cleanup-timestamp"
+  private val TIMERS_STATE_NAME = "timers"
+
+  private var latestRegisteredCleanUpTimer: ValueState[JLong] = _
+
+  protected val stateCleaningEnabled: Boolean = minRetentionTime > 1
+
+  protected var timerService: SimpleTimerService = _
+
+  override def open(): Unit = {
+
+    initializeTimerService()
+
+    if (stateCleaningEnabled) {
+      val cleanUpStateDescriptor: ValueStateDescriptor[JLong] =
+        new ValueStateDescriptor[JLong](CLEANUP_TIMESTAMP, Types.LONG)
+      latestRegisteredCleanUpTimer = getRuntimeContext.getState(cleanUpStateDescriptor)
+    }
+  }
+
+  private def initializeTimerService(): Unit = {
+    val internalTimerService = getInternalTimerService(
+      TIMERS_STATE_NAME,
+      VoidNamespaceSerializer.INSTANCE,
+      this)
+
+    timerService = new SimpleTimerService(internalTimerService)
+  }
+
+  /**
+    * If the user has specified a `minRetentionTime` and `maxRetentionTime`, this
+    * method registers a cleanup timer for `currentProcessingTime + minRetentionTime`.
+    *
+    * <p>When this timer fires, the [[BaseTwoInputStreamOperatorWithStateRetention.cleanUpState()]]
+    * method is called.
+    */
+  protected def registerProcessingCleanUpTimer(): Unit = {
+    if (stateCleaningEnabled) {
+      val currentProcessingTime = timerService.currentProcessingTime()
+      val currentCleanUpTime = Optional.ofNullable[JLong](latestRegisteredCleanUpTimer.value())
+
+      if (!currentCleanUpTime.isPresent
+        || (currentProcessingTime + minRetentionTime) > currentCleanUpTime.get) {
+
+        updateCleanUpTimer(currentProcessingTime, currentCleanUpTime)
+      }
+    }
+  }
+
+  private def updateCleanUpTimer(
+      currentProcessingTime: JLong,
+      currentCleanUpTime: Optional[JLong]): Unit = {
+
+    if (currentCleanUpTime.isPresent) {
+      timerService.deleteProcessingTimeTimer(currentCleanUpTime.get())
+    }
+
+    val newCleanUpTime: JLong = currentProcessingTime + maxRetentionTime
+    timerService.registerProcessingTimeTimer(newCleanUpTime)
+    latestRegisteredCleanUpTimer.update(newCleanUpTime)
+  }
+
+  protected def cleanUpLastTimer(): Unit = {
+    if (stateCleaningEnabled) {
+      val currentCleanUpTime = Optional.ofNullable[JLong](latestRegisteredCleanUpTimer.value())
+      if (currentCleanUpTime.isPresent) {
+        latestRegisteredCleanUpTimer.clear()
+        timerService.deleteProcessingTimeTimer(currentCleanUpTime.get())
+      }
+    }
+  }
+
+  /**
+    * The users of this class are not allowed to use processing time timers.
+    * See class javadoc.
+    */
+  override final def onProcessingTime(timer: InternalTimer[Any, VoidNamespace]): Unit = {
+    if (stateCleaningEnabled) {
+      val timerTime = timer.getTimestamp
+      val cleanupTime = latestRegisteredCleanUpTimer.value()
+
+      if (cleanupTime != null && cleanupTime == timerTime) {
+        cleanUpState(cleanupTime)
+        latestRegisteredCleanUpTimer.clear()
+      }
+    }
+  }
+
+  // ----------------- Abstract Methods -----------------
+
+  /**
+    * The method to be called when a cleanup timer fires.
+    * @param time The timestamp of the fired timer.
+    */
+  def cleanUpState(time: Long): Unit
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala
index 7a8bb61b1cf..ffda1e200af 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala
@@ -22,7 +22,8 @@ import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, TimestampedCollector, TwoInputStreamOperator}
+import org.apache.flink.runtime.state.VoidNamespace
+import org.apache.flink.streaming.api.operators.{InternalTimer, TimestampedCollector}
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
 import org.apache.flink.table.api.StreamQueryConfig
 import org.apache.flink.table.codegen.Compiler
@@ -38,8 +39,7 @@ class TemporalProcessTimeJoin(
     genJoinFuncName: String,
     genJoinFuncCode: String,
     queryConfig: StreamQueryConfig)
-  extends AbstractStreamOperator[CRow]
-  with TwoInputStreamOperator[CRow, CRow, CRow]
+  extends BaseTwoInputStreamOperatorWithStateRetention(queryConfig)
   with Compiler[FlatJoinFunction[Row, Row, Row]]
   with Logging {
 
@@ -70,30 +70,50 @@ class TemporalProcessTimeJoin(
     collector = new TimestampedCollector[CRow](output)
     cRowWrapper = new CRowWrappingCollector()
     cRowWrapper.out = collector
+
+    super.open()
   }
 
   override def processElement1(element: StreamRecord[CRow]): Unit = {
 
-    if (rightState.value() == null) {
+    val rightSideRow = rightState.value()
+    if (rightSideRow == null) {
       return
     }
 
     cRowWrapper.setChange(element.getValue.change)
 
-    val rightSideRow = rightState.value()
     joinFunction.join(element.getValue.row, rightSideRow, cRowWrapper)
+
+    registerProcessingCleanUpTimer()
   }
 
   override def processElement2(element: StreamRecord[CRow]): Unit = {
 
     if (element.getValue.change) {
       rightState.update(element.getValue.row)
+      registerProcessingCleanUpTimer()
     } else {
       rightState.clear()
+      cleanUpLastTimer()
     }
   }
 
   override def close(): Unit = {
     FunctionUtils.closeFunction(joinFunction)
   }
+
+  /**
+    * The method to be called when a cleanup timer fires.
+    *
+    * @param time The timestamp of the fired timer.
+    */
+  override def cleanUpState(time: Long): Unit = {
+    rightState.clear()
+  }
+
+  /**
+    * Invoked when an event-time timer fires.
+    */
+  override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = ???
 }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
index 66b60d44969..f6911096f93 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
@@ -26,8 +26,7 @@ import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.state._
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.runtime.state.{VoidNamespace, VoidNamespaceSerializer}
-import org.apache.flink.streaming.api.SimpleTimerService
+import org.apache.flink.runtime.state.VoidNamespace
 import org.apache.flink.streaming.api.operators._
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
 import org.apache.flink.table.api.StreamQueryConfig
@@ -69,9 +68,7 @@ class TemporalRowtimeJoin(
     queryConfig: StreamQueryConfig,
     leftTimeAttribute: Int,
     rightTimeAttribute: Int)
-  extends AbstractStreamOperator[CRow]
-  with TwoInputStreamOperator[CRow, CRow, CRow]
-  with Triggerable[Any, VoidNamespace]
+  extends BaseTwoInputStreamOperatorWithStateRetention(queryConfig)
   with Compiler[FlatJoinFunction[Row, Row, Row]]
   with Logging {
 
@@ -112,7 +109,6 @@ class TemporalRowtimeJoin(
 
   private var cRowWrapper: CRowWrappingCollector = _
   private var collector: TimestampedCollector[CRow] = _
-  private var timerService: SimpleTimerService = _
 
   private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
 
@@ -142,12 +138,7 @@ class TemporalRowtimeJoin(
     cRowWrapper.out = collector
     cRowWrapper.setChange(true)
 
-    val internalTimerService = getInternalTimerService(
-      TIMERS_STATE_NAME,
-      VoidNamespaceSerializer.INSTANCE,
-      this)
-
-    timerService = new SimpleTimerService(internalTimerService)
+    super.open()
   }
 
   override def processElement1(element: StreamRecord[CRow]): Unit = {
@@ -155,6 +146,8 @@ class TemporalRowtimeJoin(
 
     leftState.put(getNextLeftIndex, element.getValue.row)
     registerSmallestTimer(getLeftTime(element.getValue.row)) // Timer to emit and clean up the state
+
+    registerProcessingCleanUpTimer()
   }
 
   override def processElement2(element: StreamRecord[CRow]): Unit = {
@@ -163,10 +156,8 @@ class TemporalRowtimeJoin(
     val rowTime = getRightTime(element.getValue.row)
     rightState.put(rowTime, element.getValue.row)
     registerSmallestTimer(rowTime) // Timer to clean up the state
-  }
 
-  override def onProcessingTime(timer: InternalTimer[Any, VoidNamespace]): Unit = {
-    throw new IllegalStateException("This should never happen")
+    registerProcessingCleanUpTimer()
   }
 
   override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = {
@@ -175,6 +166,15 @@ class TemporalRowtimeJoin(
     if (lastUnprocessedTime < Long.MaxValue) {
       registerTimer(lastUnprocessedTime)
     }
+
+    // if we have more state at any side, then update the timer, else clean it up.
+    if (stateCleaningEnabled) {
+      if (lastUnprocessedTime < Long.MaxValue || rightState.iterator().hasNext) {
+        registerProcessingCleanUpTimer()
+      } else {
+        cleanUpLastTimer()
+      }
+    }
   }
 
   override def close(): Unit = {
@@ -245,6 +245,16 @@ class TemporalRowtimeJoin(
     }
   }
 
+  /**
+    * The method to be called when a cleanup timer fires.
+    *
+    * @param time The timestamp of the fired timer.
+    */
+  override def cleanUpState(time: Long): Unit = {
+    leftState.clear()
+    rightState.clear()
+  }
+
   private def firstIndexToKeep(timerTimestamp: Long, rightRowsSorted: util.List[Row]): Int = {
     val firstIndexNewerThenTimer =
       indexOfFirstElementNewerThanTimer(timerTimestamp, rightRowsSorted)
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
new file mode 100644
index 00000000000..e8305b77b79
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
@@ -0,0 +1,174 @@
+/*
+ * 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.harness
+
+import java.lang.{Long => JLong}
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.runtime.state.VoidNamespace
+import org.apache.flink.streaming.api.operators.InternalTimer
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness
+import org.apache.flink.table.api.StreamQueryConfig
+import org.apache.flink.table.runtime.harness.HarnessTestBase.{TestStreamQueryConfig, TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.BaseTwoInputStreamOperatorWithStateRetention
+import org.apache.flink.table.runtime.types.CRow
+import org.hamcrest.{Description, TypeSafeMatcher}
+import org.junit.{After, Before, Test}
+import org.hamcrest.MatcherAssert.assertThat
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+/**
+  * Tests for the [[BaseTwoInputStreamOperatorWithStateRetention]].
+  */
+class BaseTwoInputStreamOperatorWithStateRetentionTest extends HarnessTestBase {
+
+  private val recordAForFirstKey = new StreamRecord(CRow(1L: JLong, "hello"))
+  private val recordBForFirstKey = new StreamRecord(CRow(1L: JLong, "world"))
+
+  private val streamQueryConfig = new TestStreamQueryConfig(
+    Time.milliseconds(2),
+    Time.milliseconds(4)
+  )
+
+  private var operatorUnderTest: StubOperatorWithStateTTL = _
+
+  private var testHarness: KeyedTwoInputStreamOperatorTestHarness[JLong, CRow, CRow, CRow] = _
+
+  @Before
+  def createTestHarness(): Unit = {
+    operatorUnderTest = new StubOperatorWithStateTTL(streamQueryConfig)
+    testHarness = createTestHarness(operatorUnderTest)
+    testHarness.open()
+  }
+
+  @After
+  def closeTestHarness(): Unit = {
+    testHarness.close()
+  }
+
+  @Test
+  def normalScenarioWorks(): Unit = {
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(recordAForFirstKey)
+
+    testHarness.setProcessingTime(10L)
+
+    assertThat(operatorUnderTest, hasFiredCleanUpTimersForTimestamps(5L))
+  }
+
+  @Test
+  def whenCurrentTimePlusMinRetentionSmallerThanCurrentCleanupTimeNoNewTimerRegistered(): Unit = {
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(recordAForFirstKey)
+
+    testHarness.setProcessingTime(2L)
+    testHarness.processElement1(recordBForFirstKey)
+
+    testHarness.setProcessingTime(20L)
+
+    assertThat(operatorUnderTest, hasFiredCleanUpTimersForTimestamps(5L))
+  }
+
+  @Test
+  def whenCurrentTimePlusMinRetentionLargerThanCurrentCleanupTimeTimerIsUpdated(): Unit = {
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(recordAForFirstKey)
+
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement1(recordBForFirstKey)
+
+    testHarness.setProcessingTime(20L)
+
+    assertThat(operatorUnderTest, hasFiredCleanUpTimersForTimestamps(8L))
+  }
+
+  @Test
+  def otherSideToSameKeyStateAlsoUpdatesCleanupTimer(): Unit = {
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(recordAForFirstKey)
+
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement2(recordBForFirstKey)
+
+    testHarness.setProcessingTime(20L)
+
+    assertThat(operatorUnderTest, hasFiredCleanUpTimersForTimestamps(8L))
+  }
+
+  // -------------------------------- Test Utilities --------------------------------
+
+  private def createTestHarness(operator: BaseTwoInputStreamOperatorWithStateRetention) = {
+    new KeyedTwoInputStreamOperatorTestHarness[JLong, CRow, CRow, CRow](
+      operator,
+      new TupleRowKeySelector[JLong](0),
+      new TupleRowKeySelector[JLong](0),
+      BasicTypeInfo.LONG_TYPE_INFO,
+      1,
+      1,
+      0)
+  }
+
+  // -------------------------------- Matchers --------------------------------
+
+  private def hasFiredCleanUpTimersForTimestamps(timers: JLong*) =
+    new TypeSafeMatcher[StubOperatorWithStateTTL]() {
+
+      override protected def matchesSafely(operator: StubOperatorWithStateTTL): Boolean = {
+        operator.firedCleanUpTimers.toArray.deep == timers.toArray.deep
+      }
+
+      def describeTo(description: Description): Unit = {
+        description
+          .appendText("a list of timers with timestamps=")
+          .appendValue(timers.mkString(","))
+      }
+    }
+
+  // -------------------------------- Test Classes --------------------------------
+
+  /**
+    * A mock [[BaseTwoInputStreamOperatorWithStateRetention]] which registers
+    * the timestamps of the clean-up timers that fired (not the registered
+    * ones, which can be deleted without firing).
+    */
+  class StubOperatorWithStateTTL(
+      queryConfig: StreamQueryConfig)
+    extends BaseTwoInputStreamOperatorWithStateRetention(queryConfig) {
+
+    val firedCleanUpTimers: mutable.Buffer[JLong] = ArrayBuffer.empty
+
+    override def cleanUpState(time: Long): Unit = {
+      firedCleanUpTimers.append(time)
+    }
+
+    override def processElement1(element: StreamRecord[CRow]): Unit = {
+      registerProcessingCleanUpTimer()
+    }
+
+    override def processElement2(element: StreamRecord[CRow]): Unit = {
+      registerProcessingCleanUpTimer()
+    }
+
+    override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = ()
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
index 0d46db41970..94c22e675f0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
@@ -42,7 +42,7 @@ import org.apache.flink.table.runtime.CRowKeySelector
 import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, TestStreamQueryConfig}
 import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
-import org.hamcrest.{CoreMatchers, Matcher}
+import org.hamcrest.CoreMatchers
 import org.hamcrest.Matchers.{endsWith, startsWith}
 import org.junit.Assert.assertTrue
 import org.junit.Test
@@ -530,6 +530,210 @@ class TemporalJoinHarnessTest extends HarnessTestBase {
       0)
   }
 
+  // ---------------------- Row time TTL tests ----------------------
+
+  @Test
+  def testRowTimeJoinCleanupTimerUpdatedFromProbeSide(): Unit = {
+    // min=2ms max=4ms
+    val testHarness = createTestHarness(new OrdersRatesRowtimeTemporalJoinInfo())
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.setProcessingTime(1L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 1L)))
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, 0L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 1L, "Euro", 114L, 0L)))
+
+    testHarness.processBothWatermarks(new Watermark(2L))
+
+    // this should update the clean-up timer to 8
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 4L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 4L, "Euro", 114L, 0L)))
+
+    // this should now do nothing (also it does not update the timer as 5 + 2ms (min) < 8)
+    testHarness.setProcessingTime(5L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 5L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 5L, "Euro", 114L, 0L)))
+
+    testHarness.processBothWatermarks(new Watermark(5L))
+
+    // this should now clean up the state
+    testHarness.setProcessingTime(8L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 7L))) // this should find no match
+
+    testHarness.processBothWatermarks(new Watermark(10L))
+
+    verify(expectedOutput, testHarness.getOutput)
+
+    testHarness.close()
+  }
+
+  @Test
+  def testRowTimeJoinCleanupTimerUpdatedFromBuildSide(): Unit = {
+    // min=2ms max=4ms
+    val testHarness = createTestHarness(new OrdersRatesRowtimeTemporalJoinInfo())
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.setProcessingTime(1L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 1L)))
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, 0L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 1L, "Euro", 114L, 0L)))
+
+    testHarness.processBothWatermarks(new Watermark(2L))
+
+    // this should update the clean-up timer to 8
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 117L, 4L)))
+
+    // this should now do nothing
+    testHarness.setProcessingTime(5L)
+
+    // so this should be joined with the "old" value
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 3L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 3L, "Euro", 114L, 0L)))
+
+    testHarness.processBothWatermarks(new Watermark(5L))
+
+    // this should now clean up the state
+    testHarness.setProcessingTime(8L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 7L))) // this should find no match
+
+    testHarness.processBothWatermarks(new Watermark(10L))
+
+    verify(expectedOutput, testHarness.getOutput)
+
+    testHarness.close()
+  }
+
+  @Test
+  def testRowTimeJoinCleanupTimerUpdatedAfterEvaluation(): Unit = {
+    // min=2ms max=4ms
+    val testHarness = createTestHarness(new OrdersRatesRowtimeTemporalJoinInfo())
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.setProcessingTime(1L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 1L)))
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, 0L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 1L, "Euro", 114L, 0L)))
+
+    testHarness.setProcessingTime(4L)
+
+    // this should trigger an evaluation, which should also update the clean-up timer to 8
+    testHarness.processBothWatermarks(new Watermark(2L))
+
+    // this should now do nothing (also it does not update the timer as 5 + 2ms (min) < 8)
+    testHarness.setProcessingTime(5L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 3L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 3L, "Euro", 114L, 0L)))
+
+    testHarness.processBothWatermarks(new Watermark(5L))
+
+    // this should now clean up the state
+    testHarness.setProcessingTime(8L)
+
+    // so this should not find any match
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 7L)))
+
+    testHarness.processBothWatermarks(new Watermark(10L))
+
+    verify(expectedOutput, testHarness.getOutput)
+
+    testHarness.close()
+  }
+
+  // ---------------------- Processing time TTL tests ----------------------
+
+  @Test
+  def testProcessingTimeJoinCleanupTimerUpdatedFromProbeSide(): Unit = {
+    // min=2ms max=4ms
+    val testHarness = createTestHarness(new OrdersRatesProctimeTemporalJoinInfo())
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.setProcessingTime(1L)
+
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, 0L)))
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 1L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 1L, "Euro", 114L, 0L)))
+
+    // this should push further the clean-up the state
+    testHarness.setProcessingTime(4L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 6L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 6L, "Euro", 114L, 0L)))
+
+    // this should do nothing
+    testHarness.setProcessingTime(5L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 8L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 8L, "Euro", 114L, 0L)))
+
+    // this should clean up the state
+    testHarness.setProcessingTime(8L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 10L)))
+
+    verify(expectedOutput, testHarness.getOutput)
+
+    testHarness.close()
+  }
+
+  @Test
+  def testProcessingTimeJoinCleanupTimerUpdatedFromBuildSide(): Unit = {
+    // min=2ms max=4ms
+    val testHarness = createTestHarness(new OrdersRatesProctimeTemporalJoinInfo())
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.setProcessingTime(1L)
+
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, 0L)))
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 1L)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 1L, "Euro", 114L, 0L)))
+
+    // this should push further the clean-up the state
+    testHarness.setProcessingTime(4L)
+
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 116L, 1L)))
+
+    // this should do nothing
+    testHarness.setProcessingTime(5L)
+
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 6L)))
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 6L, "Euro", 116L, 1L)))
+
+    // this should clean up the state
+    testHarness.setProcessingTime(8L)
+
+    // so this should find no match
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 10L)))
+
+    verify(expectedOutput, testHarness.getOutput)
+
+    testHarness.close()
+  }
+
   def translateJoin(joinInfo: TemporalJoinInfo, joinRelType: JoinRelType = JoinRelType.INNER)
     : (CRowKeySelector, CRowKeySelector, TwoInputStreamOperator[CRow, CRow, CRow]) = {
 
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
index 7bb697331da..9289252479a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
@@ -64,4 +64,9 @@ public void processWatermark1(Watermark mark) throws Exception {
 	public void processWatermark2(Watermark mark) throws Exception {
 		twoInputOperator.processWatermark2(mark);
 	}
+
+	public void processBothWatermarks(Watermark mark) throws Exception {
+		twoInputOperator.processWatermark1(mark);
+		twoInputOperator.processWatermark2(mark);
+	}
 }


 

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


With regards,
Apache Git Services