[
https://issues.apache.org/jira/browse/FLINK-10583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16703260#comment-16703260
]
ASF GitHub Bot commented on FLINK-10583:
----------------------------------------
pnowojski commented on a change in pull request #6871:
[FLINK-10583][FLINK-10584][table] Add State Retention to temporal joins.
URL: https://github.com/apache/flink/pull/6871#discussion_r237505985
##########
File path:
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
##########
@@ -530,6 +530,214 @@ class TemporalJoinHarnessTest extends HarnessTestBase {
0)
}
+ // ---------------------- Event time TTL tests ----------------------
+
+ @Test
+ def testEventTimeScenarioWithoutAdvancingProcessingTime(): Unit = {
+ // min=2ms max=4ms
+ val testHarness = createTestHarness(new
OrdersRatesRowtimeTemporalJoinInfo())
+
+ testHarness.open()
+ val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+ testHarness.setProcessingTime(1L)
+
+ // process without conversion rates
+ 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.processWatermark1(new Watermark(2L))
+ testHarness.processWatermark2(new Watermark(2L))
+
+ verify(expectedOutput, testHarness.getOutput)
+
+ testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", 4L)))
+
+ expectedOutput.add(new StreamRecord(CRow(2L, "Euro", 4L, "Euro", 114L,
0L)))
+
+ testHarness.processWatermark1(new Watermark(5L))
+ testHarness.processWatermark2(new Watermark(5L))
+
+ verify(expectedOutput, testHarness.getOutput)
+
+ testHarness.close()
+ }
+
+ @Test
+ def testEventTimeCleanupShouldSucceed(): Unit = {
+ // min=2ms max=4ms
+ val testHarness = createTestHarness(new
OrdersRatesRowtimeTemporalJoinInfo())
+
+ testHarness.open()
+ val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+ testHarness.setProcessingTime(1L)
+
+ // process without conversion rates
+ 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.processWatermark1(new Watermark(2L))
+ testHarness.processWatermark2(new Watermark(2L))
Review comment:
Now that I think about it, I would add:
```
public void processWatermarks(Watermark mark) throws Exception {
twoInputOperator.processWatermark1(mark);
twoInputOperator.processWatermark2(mark);
}
```
method to the `TwoInputStreamOperatorTestHarness`. This pattern of advancing
both watermarks at the same time is duplicated in multiple (if not in all)
places...
----------------------------------------------------------------
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:
[email protected]
> 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.8.0
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)