[
https://issues.apache.org/jira/browse/BEAM-5197?focusedWorklogId=169975&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-169975
]
ASF GitHub Bot logged work on BEAM-5197:
----------------------------------------
Author: ASF GitHub Bot
Created on: 27/Nov/18 19:29
Start Date: 27/Nov/18 19:29
Worklog Time Spent: 10m
Work Description: tweise commented on a change in pull request #7138:
[BEAM-5197] Increase test time progressing interval to avoid deadlock
URL: https://github.com/apache/beam/pull/7138#discussion_r236812891
##########
File path:
runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
##########
@@ -300,21 +304,37 @@ public void close() {}
sourceThread.start();
- while (true) {
- if (!caughtExceptions.isEmpty()) {
- fail("Caught exception(s): " +
Joiner.on(",").join(caughtExceptions));
- }
- if (seenWatermark.get()) {
- break;
- }
+ while (flinkWrapper
+ .getLocalReaders()
+ .stream()
+ .anyMatch(reader -> reader.getWatermark().getMillis() == 0)) {
+ // readers haven't been initialized
Thread.sleep(50);
+ }
- // Need to advance this so that the watermark timers in the source
wrapper fire
- // Synchronize is necessary because this can interfere with updating
the PriorityQueue
- // of the ProcessingTimeService which is also accessed through
UnboundedSourceWrapper.
- synchronized (checkpointLock) {
- testHarness.setProcessingTime(Instant.now().getMillis());
- }
+ // Need to advance this so that the watermark timers in the source
wrapper fire
+ // Synchronize is necessary because this can interfere with updating the
PriorityQueue
+ // of the ProcessingTimeService which is also accessed through
UnboundedSourceWrapper.
+ synchronized (testHarness.getCheckpointLock()) {
Review comment:
This can be moved up to initialization and synchronized won't be needed then.
----------------------------------------------------------------
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 169975)
Time Spent: 5h 10m (was: 5h)
> Flaky test: UnboundedSourceWrapperTest$ParameterizedUnboundedSourceWrapperTest
> ------------------------------------------------------------------------------
>
> Key: BEAM-5197
> URL: https://issues.apache.org/jira/browse/BEAM-5197
> Project: Beam
> Issue Type: Improvement
> Components: runner-flink
> Reporter: Thomas Weise
> Assignee: Maximilian Michels
> Priority: Critical
> Labels: flake
> Fix For: 2.10.0
>
> Time Spent: 5h 10m
> Remaining Estimate: 0h
>
> {code:java}
> java.lang.NullPointerException
> at
> org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService$1.compare(TestProcessingTimeService.java:52)
> at
> org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService$1.compare(TestProcessingTimeService.java:49)
> at java.util.PriorityQueue.siftUpUsingComparator(PriorityQueue.java:670)
> at java.util.PriorityQueue.siftUp(PriorityQueue.java:646)
> at java.util.PriorityQueue.offer(PriorityQueue.java:345)
> at
> org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService.registerTimer(TestProcessingTimeService.java:93)
> at
> org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.setNextWatermarkTimer(UnboundedSourceWrapper.java:452)
> at
> org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.run(UnboundedSourceWrapper.java:225)
> at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:87)
> at
> org.apache.beam.runners.flink.streaming.UnboundedSourceWrapperTest$ParameterizedUnboundedSourceWrapperTest.testValueEmission(UnboundedSourceWrapperTest.java:153)
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)