[
https://issues.apache.org/jira/browse/FLINK-4552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15614968#comment-15614968
]
ASF GitHub Bot commented on FLINK-4552:
---------------------------------------
Github user aljoscha commented on the issue:
https://github.com/apache/flink/pull/2572
No, it's more or less that but there is more stuff that needs to be in that
adapter, for example in `testOnProcessingTimeFire()`, I'm highlighting the
places that need changing:
```
public void testOnProcessingTimeFire() throws Exception {
WindowAssigner<Integer, TimeWindow> mockAssigner =
mockTimeWindowAssigner();
when(mockAssigner.isEventTime()).thenReturn(false); <-- here
Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
KeyedOneInputStreamOperatorTestHarness<Integer, Integer,
WindowedValue<List<Integer>, TimeWindow>> testHarness =
createListWindowOperator(mockAssigner, mockTrigger, 0L);
testHarness.open();
testHarness.setProcessingTime(Long.MIN_VALUE); <-- here
when(mockAssigner.assignWindows(anyInt(), anyLong(),
anyAssignerContext()))
.thenReturn(Arrays.asList(new TimeWindow(2, 4), new
TimeWindow(0, 2)));
assertEquals(0, testHarness.getOutput().size());
assertEquals(0, testHarness.numKeyedStateEntries());
doAnswer(new Answer<TriggerResult>() {
@Override
public TriggerResult answer(InvocationOnMock invocation) throws
Exception {
Trigger.TriggerContext context = (Trigger.TriggerContext)
invocation.getArguments()[3];
context.registerProcessingTimeTimer(0L); <-- here
context.getPartitionedState(valueStateDescriptor).update("hello");
return TriggerResult.CONTINUE;
}
}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(),
anyLong(), anyTimeWindow(), anyTriggerContext());
shouldFireOnProcessingTime(mockTrigger); <-- here
testHarness.processElement(new StreamRecord<>(0, 0L));
assertEquals(4, testHarness.numKeyedStateEntries()); // window-contents
and trigger state for two windows
assertEquals(4, testHarness.numProcessingTimeTimers()); // timers/gc
timers for two windows <-- here
testHarness.setProcessingTime(0L); <-- here
// clear is only called at cleanup time/GC time
verify(mockTrigger, never()).clear(anyTimeWindow(),
anyTriggerContext());
// FIRE should not purge contents
assertEquals(4, testHarness.numKeyedStateEntries());
assertEquals(2, testHarness.numProcessingTimeTimers()); // only gc
timers left <-- here
// there should be two elements now
assertThat(testHarness.extractOutputStreamRecords(),
containsInAnyOrder(
isWindowedValue(contains(0), 1L, timeWindow(0, 2)),
isWindowedValue(contains(0), 3L, timeWindow(2, 4))));
}
```
(man I can't make text bold in a code block ... ðŸ˜)
> Refactor WindowOperator/Trigger Tests
> -------------------------------------
>
> Key: FLINK-4552
> URL: https://issues.apache.org/jira/browse/FLINK-4552
> Project: Flink
> Issue Type: Improvement
> Components: Windowing Operators
> Reporter: Aljoscha Krettek
> Assignee: Aljoscha Krettek
>
> Right now, tests for {{WindowOperator}}, {{WindowAssigner}}, {{Trigger}} and
> {{WindowFunction}} are all conflated in {{WindowOperatorTest}}. All of these
> test that a certain combination of a {{Trigger}}, {{WindowAssigner}} and
> {{WindowFunction}} produce the expected output.
> We should modularize these tests and spread them out across multiple files,
> possibly one per trigger, for the triggers. Also, we should extend/change the
> tests in some key ways:
> - {{WindowOperatorTest}} test should just verify that the interaction
> between {{WindowOperator}} and the various other parts works as expected,
> that the correct methods on {{Trigger}} and {{WindowFunction}} are called at
> the expected time and that snapshotting, timers, cleanup etc. work correctly.
> These tests should also verify that the different state types and
> {{WindowFunctions}} work correctly.
> - {{Trigger}} tests should present elements to triggers and verify that they
> fire at the correct times. The actual output of the {{WindowFunction}} is not
> important for these tests. We should also test that triggers correctly clean
> up state and timers.
> - {{WindowAssigner}} tests should test each window assigner and also verify
> that, for example, the offset parameter of time-based windows works correctly.
> There is already {{WindowingTestHarness}} but it is not used by tests, I
> think we can expand on that and provide more thorough test coverage while
> also making the tests more maintainable ({{WindowOperatorTest.java}} is
> nearing 3000 lines of code).
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)