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

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

pnowojski commented on a change in pull request #6871: [FLINK-10583][table] Add 
base TwoInputStreamOperator with TTL operator.
URL: https://github.com/apache/flink/pull/6871#discussion_r225965180
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
 ##########
 @@ -0,0 +1,332 @@
+/*
+ * 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.state._
+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.api.scala.OutputTag
+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.AbstractTwoInputStreamOperatorWithTTL
+import org.apache.flink.table.runtime.types.CRow
+import org.hamcrest.{Description, TypeSafeMatcher}
+import org.junit.Test
+import org.hamcrest.MatcherAssert.assertThat
+
+import scala.collection.JavaConverters._
+import org.apache.flink.api.scala._
+
+/**
+  * Tests for the [[AbstractTwoInputStreamOperatorWithTTL]].
+  */
+class AbstractTwoInputStreamOperatorWithTTLTest extends HarnessTestBase {
+
+  private var testHarness: KeyedTwoInputStreamOperatorTestHarness[JLong, CRow, 
CRow, CRow] = _
+
+  private val streamQueryConfig = new TestStreamQueryConfig(
+    Time.milliseconds(2),
+    Time.milliseconds(4)
+  )
+
+  @Test
+  def normalScenarioWorks(): Unit = {
+    val operator: StubOperatorWithTTLTimers =
+      new StubOperatorWithTTLTimers(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 1))
+
+    testHarness.setProcessingTime(10L)
+    testHarness.close()
+
+    assertThat(operator, hasFiredTimers(5L))
+  }
+
+  @Test
+  def 
whenCurrentTimePlusMinRetentionSmallerThanCurrentCleanupTimeNoNewTimerRegistered():
 Unit = {
+    val operator: StubOperatorWithTTLTimers =
+      new StubOperatorWithTTLTimers(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 1))
+
+    testHarness.setProcessingTime(2L)
+    testHarness.processElement2(new StreamRecord(CRow(1L: JLong, "world"), 1))
+
+    testHarness.setProcessingTime(20L)
+    testHarness.close()
+
+    assertThat(operator, hasFiredTimers(5L))
+  }
+
+  @Test
+  def 
whenCurrentTimePlusMinRetentionLargerThanCurrentCleanupTimeTimerIsUpdated(): 
Unit = {
+    val operator: StubOperatorWithTTLTimers =
+      new StubOperatorWithTTLTimers(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 1))
+
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "world"), 1))
+
+    testHarness.setProcessingTime(20L)
+    testHarness.close()
+
+    assertThat(operator, hasFiredTimers(8L))
+  }
+
+  @Test
+  def otherSideToSameKeyStateAlsoUpdatesCleanupTimer(): Unit = {
+    val operator: StubOperatorWithTTLTimers =
+      new StubOperatorWithTTLTimers(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(1L)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 1))
+
+    testHarness.setProcessingTime(4L)
+    testHarness.processElement2(new StreamRecord(CRow(1L: JLong, "world"), 1))
+
+    testHarness.setProcessingTime(20L)
+    testHarness.close()
+
+    assertThat(operator, hasFiredTimers(8L))
+  }
+
+  @Test
+  def testTimerActuallyCleansUpTheState() {
+    val operator: StubOperatorWithStateTTL =
+      new StubOperatorWithStateTTL(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(1)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 1))
+
+    testHarness.setProcessingTime(2)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "beautiful"), 
2))
+
+    // here we will have the first timer firing
+    testHarness.setProcessingTime(10L)
+
+    assertThat(
+      operator,
+      hasCleanedUpState(
+        new StreamRecord[(JLong, List[String])]((1L, List("hello", 
"beautiful")))))
+
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "world"), 10))
+
+    // and here the second one
+    testHarness.setProcessingTime(20L)
+
+    assertThat(
+      operator,
+      hasCleanedUpState(
+        new StreamRecord[(JLong, List[String])]((1L, List("hello", 
"beautiful"))),
+        new StreamRecord[(JLong, List[String])]((1L, List("world")))))
+
+    testHarness.close()
+  }
+
+  @Test
+  def testTimerCleansUpOnlyStateOfAssociatedKey(): Unit = {
+    val operator: StubOperatorWithStateTTL =
+      new StubOperatorWithStateTTL(streamQueryConfig)
+
+    testHarness = createTestHarness(operator)
+
+    testHarness.open()
+
+    testHarness.setProcessingTime(2)
+    testHarness.processElement1(new StreamRecord(CRow(1L: JLong, "hello"), 2))
+
+    testHarness.setProcessingTime(4)
+    testHarness.processElement1(new StreamRecord(CRow(2L: JLong, "beautiful"), 
10))
+
+    // here we will have the first timer firing
+    testHarness.setProcessingTime(5)
+
+    testHarness.processElement2(new StreamRecord(CRow(2L: JLong, "world"), 10))
+
+    // and here the second one
+    testHarness.setProcessingTime(20L)
+
+    assertThat(
+      operator,
+      hasOutput(new StreamRecord[CRow](CRow(2L, List("beautiful", "world")))))
+
+    testHarness.close()
+  }
+
+  // -------------------------------- Test Utilities 
--------------------------------
+
+  private def createTestHarness(operator: 
AbstractTwoInputStreamOperatorWithTTL) = {
+    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 hasOutput (expected: Object*) =
+    new TypeSafeMatcher[StubOperatorWithStateTTL]() {
+
+      override def matchesSafely(operator: StubOperatorWithStateTTL): Boolean 
= {
+        expected.toArray.deep == testHarness.getOutput().toArray.deep
+      }
+
+      override def describeTo(description: Description): Unit = {
+        description
+          .appendText("main output=")
+          .appendValue(expected.toArray.deep.mkString(", "))
+      }
+    }
+
+  private def hasCleanedUpState (expected: Object*) =
+    new TypeSafeMatcher[StubOperatorWithStateTTL]() {
+
+      override def matchesSafely(operator: StubOperatorWithStateTTL): Boolean 
= {
+        expected.toArray.deep == 
testHarness.getSideOutput(operator.outputTag).toArray.deep
+      }
+
+      override def describeTo(description: Description): Unit = {
+        description
+          .appendText("cleaned element output=")
+          .appendValue(expected.toArray.deep.mkString(", "))
+      }
+    }
+
+  private def hasFiredTimers(timers: JLong*) =
+    new TypeSafeMatcher[StubOperatorWithTTLTimers]() {
+
+      override protected def matchesSafely(operator: 
StubOperatorWithTTLTimers): Boolean = {
+        operator.firedTimers.toArray.deep == timers.toArray.deep
+      }
+
+      def describeTo(description: Description): Unit = {
+        description
+          .appendText("a list of timers with timestamps=")
+          .appendValue(timers.mkString(","))
+      }
+    }
+
+  // -------------------------------- Test Classes 
--------------------------------
+
+  /**
+    * A stateful [[AbstractTwoInputStreamOperatorWithTTL]] which for every 
element on
+    * side 1, it appends it to a list state, while for elements on side 2, it
+    * emits the element appended to elements in the list state.
+    *
+    * In addition, it cleans up the state, so this allows to check which state
+    * was cleaned up and which remained.
+    */
+  class StubOperatorWithStateTTL(
 
 Review comment:
   Maybe instead of implementing the full operator here with a state, just 
capture in `cleanupState` arguments and in the tests assert how many times 
method was called with what arguments? It would simplify the test and I think 
would same test coverage.
   
   With modified `hasFiredTimers` that would check also for invocations of 
`cleanUpState` methods (don't use mockito for that ;) ), we probably could drop 
`StubOperatorWithStateTTL` and both tests `testTimerActuallyCleansUpTheState` 
`testTimerCleansUpOnlyStateOfAssociatedKey` would be already covered by the 
previous tests.

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


> Add support for state retention to the Processing Time versioned joins.
> -----------------------------------------------------------------------
>
>                 Key: FLINK-10583
>                 URL: https://issues.apache.org/jira/browse/FLINK-10583
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>    Affects Versions: 1.7.0
>            Reporter: Kostas Kloudas
>            Assignee: Kostas Kloudas
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>




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

Reply via email to