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

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

                Author: ASF GitHub Bot
            Created on: 23/Jul/20 21:20
            Start Date: 23/Jul/20 21:20
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#11924:
URL: https://github.com/apache/beam/pull/11924#discussion_r459729767



##########
File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
##########
@@ -24,18 +24,11 @@
 import com.google.api.services.dataflow.model.SideInputInfo;
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;

Review comment:
       Please keep the imports all explicit.

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -3950,13 +3950,39 @@ public void testEventTimeTimerOrderingWithCreate() 
throws Exception {
       }
 
       testEventTimeTimerOrderingWithInputPTransform(
-          now, numTestElements, Create.timestamped(elements));
+          now, numTestElements, Create.timestamped(elements), false);
+    }
+
+    /**
+     * A test makes sure that an event time timers are correctly ordered using 
Create transform
+     * unbounded.
+     */
+    @Test
+    @Category({
+      ValidatesRunner.class,
+      UsesTimersInParDo.class,
+      UsesStatefulParDo.class,
+      UsesUnboundedPCollections.class,
+      UsesStrictTimerOrdering.class
+    })
+    public void testEventTimeTimerOrderingWithCreateUnbounded() throws 
Exception {
+      final int numTestElements = 100;
+      final Instant now = new Instant(1500000000000L);
+
+      List<TimestampedValue<KV<String, String>>> elements = new ArrayList<>();
+      for (int i = 0; i < numTestElements; i++) {
+        elements.add(TimestampedValue.of(KV.of("dummy", "" + i), now.plus(i)));
+      }
+
+      testEventTimeTimerOrderingWithInputPTransform(
+          now, numTestElements, Create.timestamped(elements), true);
     }
 
     private void testEventTimeTimerOrderingWithInputPTransform(
         Instant now,
         int numTestElements,
-        PTransform<PBegin, PCollection<KV<String, String>>> transform)
+        PTransform<PBegin, PCollection<KV<String, String>>> transform,
+        boolean isStreaming)

Review comment:
       This doesn't depend on streaming or not, but just controls whether the 
pcollection should be bounded or unbounded. For clarity, you can just make this 
parameter `IsBounded isBounded`

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -4307,18 +4346,26 @@ public void onTimer(
     private static class TwoTimerTest extends PTransform<PBegin, PDone> {
 
       private static PTransform<PBegin, PDone> of(
-          Instant start, Instant end, PTransform<PBegin, PCollection<KV<Void, 
Void>>> input) {
-        return new TwoTimerTest(start, end, input);
+          Instant start,
+          Instant end,
+          PTransform<PBegin, PCollection<KV<Void, Void>>> input,
+          boolean isStreaming) {
+        return new TwoTimerTest(start, end, input, isStreaming);
       }
 
       private final Instant start;
       private final Instant end;
+      private final boolean isStreaming;
       private final transient PTransform<PBegin, PCollection<KV<Void, Void>>> 
inputPTransform;
 
       public TwoTimerTest(
-          Instant start, Instant end, PTransform<PBegin, PCollection<KV<Void, 
Void>>> input) {
+          Instant start,
+          Instant end,
+          PTransform<PBegin, PCollection<KV<Void, Void>>> input,
+          boolean isStreaming) {

Review comment:
       same here

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -4307,18 +4346,26 @@ public void onTimer(
     private static class TwoTimerTest extends PTransform<PBegin, PDone> {
 
       private static PTransform<PBegin, PDone> of(
-          Instant start, Instant end, PTransform<PBegin, PCollection<KV<Void, 
Void>>> input) {
-        return new TwoTimerTest(start, end, input);
+          Instant start,
+          Instant end,
+          PTransform<PBegin, PCollection<KV<Void, Void>>> input,
+          boolean isStreaming) {
+        return new TwoTimerTest(start, end, input, isStreaming);
       }
 
       private final Instant start;
       private final Instant end;
+      private final boolean isStreaming;

Review comment:
       same here

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -4331,6 +4378,7 @@ public PDone expand(PBegin input) {
         PCollection<String> result =
             input
                 .apply(inputPTransform)
+                .setIsBoundedInternal(isStreaming ? IsBounded.UNBOUNDED : 
IsBounded.BOUNDED)

Review comment:
       Another way to do this that might be better is to use `TestStream` in 
the unbounded case. This will probably give best coverage. Even for an 
unbounded PCollection the watermark might instantly move to infinity.

##########
File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
##########
@@ -561,28 +554,64 @@ public void flushState() {
       return nextTimer;
     }
 
-    // Lazily initialized
-    private Iterator<TimerData> cachedFiredUserTimers = null;
+    private PriorityQueue<TimerData> toBeFiredTimersOrdered = null;
+
+    // to track if timer is reset earlier mid-bundle.

Review comment:
       Can you add a comment about what are the keys and values of this map?

##########
File path: 
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
##########
@@ -4307,18 +4346,26 @@ public void onTimer(
     private static class TwoTimerTest extends PTransform<PBegin, PDone> {
 
       private static PTransform<PBegin, PDone> of(
-          Instant start, Instant end, PTransform<PBegin, PCollection<KV<Void, 
Void>>> input) {
-        return new TwoTimerTest(start, end, input);
+          Instant start,
+          Instant end,
+          PTransform<PBegin, PCollection<KV<Void, Void>>> input,
+          boolean isStreaming) {

Review comment:
       same here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 462706)
    Time Spent: 1h 40m  (was: 1.5h)

> Dataflow streaming timers are not strictly time ordered when set earlier 
> mid-bundle
> -----------------------------------------------------------------------------------
>
>                 Key: BEAM-8543
>                 URL: https://issues.apache.org/jira/browse/BEAM-8543
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-dataflow
>    Affects Versions: 2.13.0
>            Reporter: Jan Lukavský
>            Assignee: Rehman Murad Ali
>            Priority: P2
>              Labels: stale-assigned
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> Let's suppose we have the following situation:
>  - statful ParDo with two timers - timerA and timerB
>  - timerA is set for window.maxTimestamp() + 1
>  - timerB is set anywhere between <windowStart, windowEnd), let's denote that 
> timerB.timestamp
>  - input watermark moves to BoundedWindow.TIMESTAMP_MAX_VALUE
> Then the order of timers is as follows (correct):
>  - timerB
>  - timerA
> But, if timerB sets another timer (say for timerB.timestamp + 1), then the 
> order of timers will be:
>  - timerB (timerB.timestamp)
>  - timerA (BoundedWindow.TIMESTAMP_MAX_VALUE)
>  - timerB (timerB.timestamp + 1)
> Which is not ordered by timestamp. The reason for this is that when the input 
> watermark update is evaluated, the WatermarkManager,extractFiredTimers() will 
> produce both timerA and timerB. That would be correct, but when timerB sets 
> another timer, that breaks this.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to