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

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

                Author: ASF GitHub Bot
            Created on: 13/Oct/21 19:03
            Start Date: 13/Oct/21 19:03
    Worklog Time Spent: 10m 
      Work Description: laraschmidt commented on a change in pull request 
#15540:
URL: https://github.com/apache/beam/pull/15540#discussion_r728361761



##########
File path: 
runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
##########
@@ -1190,13 +1195,24 @@ public Timer withOutputTimestamp(Instant 
outputTimestamp) {
      * </ul>
      */
     private void setAndVerifyOutputTimestamp() {
-
       if (outputTimestamp != null) {
-        checkArgument(
-            !outputTimestamp.isBefore(elementInputTimestamp),
-            "output timestamp %s should be after input message timestamp or 
output timestamp of firing timers %s",
-            outputTimestamp,
-            elementInputTimestamp);
+        Instant lowerBound;
+        Instant upperBound = BoundedWindow.TIMESTAMP_MAX_VALUE;
+        try {
+          lowerBound = 
elementInputTimestamp.minus(fn.getAllowedTimestampSkew());
+        } catch (ArithmeticException e) {
+          lowerBound = BoundedWindow.TIMESTAMP_MIN_VALUE;
+        }
+        if (outputTimestamp.isBefore(lowerBound) || 
outputTimestamp.isAfter(upperBound)) {
+          throw new IllegalArgumentException(
+              String.format(
+                  "output timestamp %s (allowed skew %s) should be after input 
message timestamp or"
+                      + " output timestamp of firing timers %s and before %s",
+                  outputTimestamp,
+                  
PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod()),
+                  elementInputTimestamp,
+                  upperBound));

Review comment:
       Done.

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1897,14 +1898,26 @@ private Instant minTargetAndGcTime(Instant target) {
       return Timer.cleared(userKey, dynamicTimerTag, 
Collections.singletonList(boundedWindow));
     }
 
+    @SuppressWarnings("deprecation") // Allowed Skew is deprecated for users, 
but must be respected
     private Timer<K> getTimerForTime(Instant scheduledTime) {
       if (outputTimestamp != null) {
-        checkArgument(
-            !outputTimestamp.isBefore(elementTimestampOrTimerHoldTimestamp),
-            "output timestamp %s should be after input message timestamp or 
output timestamp of"
-                + " firing timers %s",
-            outputTimestamp,
-            elementTimestampOrTimerHoldTimestamp);
+        Instant lowerBound;
+        Instant upperBound = BoundedWindow.TIMESTAMP_MAX_VALUE;
+        try {
+          lowerBound = 
elementTimestampOrTimerHoldTimestamp.minus(doFn.getAllowedTimestampSkew());
+        } catch (ArithmeticException e) {
+          lowerBound = BoundedWindow.TIMESTAMP_MIN_VALUE;
+        }
+        if (outputTimestamp.isBefore(lowerBound) || 
outputTimestamp.isAfter(upperBound)) {

Review comment:
       done

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1999,6 +2012,28 @@ public void set(String dynamicTimerTag, Instant 
absoluteTime) {
     }
   }
 
+  @SuppressWarnings("deprecation") // Allowed Skew is deprecated for users, 
but must be respected
+  private void checkTimestamp(Instant timestamp) {
+    Instant lowerBound;
+    Instant upperBound = BoundedWindow.TIMESTAMP_MAX_VALUE;
+    try {
+      lowerBound = 
currentElement.getTimestamp().minus(doFn.getAllowedTimestampSkew());
+    } catch (ArithmeticException e) {
+      lowerBound = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+    if (timestamp.isBefore(lowerBound) || timestamp.isAfter(upperBound)) {

Review comment:
       done

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1999,6 +2012,28 @@ public void set(String dynamicTimerTag, Instant 
absoluteTime) {
     }
   }
 
+  @SuppressWarnings("deprecation") // Allowed Skew is deprecated for users, 
but must be respected
+  private void checkTimestamp(Instant timestamp) {
+    Instant lowerBound;
+    Instant upperBound = BoundedWindow.TIMESTAMP_MAX_VALUE;
+    try {
+      lowerBound = 
currentElement.getTimestamp().minus(doFn.getAllowedTimestampSkew());
+    } catch (ArithmeticException e) {
+      lowerBound = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+    if (timestamp.isBefore(lowerBound) || timestamp.isAfter(upperBound)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Cannot output with timestamp %s. Output timestamps must be no 
earlier than the "
+                  + "timestamp of the current input (%s) minus the allowed 
skew (%s). See the "

Review comment:
       fixed




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 665292)
    Time Spent: 3h 50m  (was: 3h 40m)

> Timer.setOutputTimestamp doesn't take into account for 
> DoFn#getAllowedTimestampSkew()
> -------------------------------------------------------------------------------------
>
>                 Key: BEAM-12931
>                 URL: https://issues.apache.org/jira/browse/BEAM-12931
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model
>            Reporter: Lara Schmidt
>            Assignee: Lara Schmidt
>            Priority: P2
>          Time Spent: 3h 50m
>  Remaining Estimate: 0h
>
> A DoFn may emit elements with a timestamp up to 
> DoFn#getAllowedTimestampSkew() before the current element's timestamp. 
> However getAllowedTimestampSkew is not properly accounted for in looking at 
> the output timestamp of a timer.
> Context: 
> [https://lists.apache.org/thread.html/r7554658114ddde86c5d82e1c39fe7e1ef587fe926b8e406d1130d501%40%3Cdev.beam.apache.org%3E]



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

Reply via email to