[ 
https://issues.apache.org/jira/browse/BEAM-5197?focusedWorklogId=169012&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-169012
 ]

ASF GitHub Bot logged work on BEAM-5197:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 23/Nov/18 10:29
            Start Date: 23/Nov/18 10:29
    Worklog Time Spent: 10m 
      Work Description: mxm closed pull request #7106: [BEAM-5197] Fix flaky 
test UnboundedSourceWrapper#testWatermarkEmission
URL: https://github.com/apache/beam/pull/7106
 
 
   

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/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
index 48699444505..97aa84ecfce 100644
--- 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
@@ -148,7 +148,7 @@ public void run() {
                 while (true) {
                   try {
                     testHarness.setProcessingTime(System.currentTimeMillis());
-                    Thread.sleep(1000);
+                    Thread.sleep(100);
                   } catch (InterruptedException e) {
                     // this is ok
                     break;
@@ -305,10 +305,14 @@ public void close() {}
         if (seenWatermark.get()) {
           break;
         }
-        Thread.sleep(10);
+        Thread.sleep(50);
 
-        // need to advance this so that the watermark timers in the source 
wrapper fire
-        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 (checkpointLock) {
+          testHarness.setProcessingTime(Instant.now().getMillis());
+        }
       }
 
       sourceOperator.cancel();


 

----------------------------------------------------------------
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: 169012)
    Time Spent: 0.5h  (was: 20m)

> 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: Major
>             Fix For: 2.10.0
>
>          Time Spent: 0.5h
>  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)

Reply via email to