beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4653) Java SDK harness should support user timers
Date Thu, 19 Jul 2018 01:03:00 GMT

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

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

                Author: ASF GitHub Bot
            Created on: 19/Jul/18 01:02
            Start Date: 19/Jul/18 01:02
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #5898: [BEAM-4653]
Add support to the Java SDK harness to execute timers.
URL: https://github.com/apache/beam/pull/5898#discussion_r203576143
 
 

 ##########
 File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
 ##########
 @@ -256,6 +287,125 @@ public void finishBundle() {
     }
   }
 
+  private class FnApiTimer implements org.apache.beam.sdk.state.Timer {
+    private final String timerId;
+    private final TimeDomain timeDomain;
+    private final Instant currentTimestamp;
+    private final Duration allowedLateness;
+    private final WindowedValue<?> currentElementOrTimer;
+
+    private Duration period = Duration.ZERO;
+    private Duration offset = Duration.ZERO;
+
+    FnApiTimer(String timerId, WindowedValue<KV<?, ?>> currentElementOrTimer)
{
+      this.timerId = timerId;
+      this.currentElementOrTimer = currentElementOrTimer;
+
+      TimerDeclaration timerDeclaration = context.doFnSignature.timerDeclarations().get(timerId);
+      this.timeDomain =
+          DoFnSignatures.getTimerSpecOrThrow(timerDeclaration, context.doFn).getTimeDomain();
+
+      switch (timeDomain) {
+        case EVENT_TIME:
+          this.currentTimestamp = currentElementOrTimer.getTimestamp();
+          break;
+        case PROCESSING_TIME:
+          this.currentTimestamp = new Instant(DateTimeUtils.currentTimeMillis());
+          break;
+        case SYNCHRONIZED_PROCESSING_TIME:
+          this.currentTimestamp = new Instant(DateTimeUtils.currentTimeMillis());
+          break;
+        default:
+          throw new IllegalStateException(String.format("Unknown timedomain %s", timeDomain));
+      }
+
+      try {
+        this.allowedLateness =
+            context
+                .rehydratedComponents
+                .getPCollection(context.pTransform.getInputsOrThrow(timerId))
+                .getWindowingStrategy()
+                .getAllowedLateness();
+      } catch (IOException e) {
+        throw new IllegalStateException(
+            String.format("Unable to get allowed lateness for timer %s", timerId));
+      }
+    }
+
+    @Override
+    public void set(Instant absoluteTime) {
+      // Verifies that the time domain of this timer is acceptable for absolute timers.
+      if (!TimeDomain.EVENT_TIME.equals(timeDomain)) {
+        throw new IllegalStateException(
+            "Can only set relative timers in processing time domain. Use #setRelative()");
+      }
+
+      // Ensures that the target time is reasonable. For event time timers this means that
the time
+      // should be prior to window GC time.
+      if (TimeDomain.EVENT_TIME.equals(timeDomain)) {
+        Instant windowExpiry = LateDataUtils.garbageCollectionTime(currentWindow, allowedLateness);
+        checkArgument(
+            !absoluteTime.isAfter(windowExpiry),
+            "Attempted to set event time timer for %s but that is after"
+                + " the expiration of window %s",
+            absoluteTime,
+            windowExpiry);
+      }
+
+      output(absoluteTime);
+    }
+
+    @Override
+    public void setRelative() {
+      Instant target;
+      if (period.equals(Duration.ZERO)) {
+        target = currentTimestamp.plus(offset);
+      } else {
+        long millisSinceStart = currentTimestamp.plus(offset).getMillis() % period.getMillis();
+        target =
+            millisSinceStart == 0
+                ? currentTimestamp
+                : currentTimestamp.plus(period).minus(millisSinceStart);
+      }
+      target = minTargetAndGcTime(target);
+      output(target);
+    }
+
+    @Override
+    public org.apache.beam.sdk.state.Timer offset(Duration offset) {
+      this.offset = offset;
+      return this;
+    }
+
+    @Override
+    public org.apache.beam.sdk.state.Timer align(Duration period) {
+      this.period = period;
+      return this;
+    }
+
+    /**
+     * For event time timers the target time should be prior to window GC time. So it return
 
 Review comment:
   fixed

----------------------------------------------------------------
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:
users@infra.apache.org


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

    Worklog Id:     (was: 124731)
    Time Spent: 2h 50m  (was: 2h 40m)

> Java SDK harness should support user timers
> -------------------------------------------
>
>                 Key: BEAM-4653
>                 URL: https://issues.apache.org/jira/browse/BEAM-4653
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-harness
>            Reporter: Luke Cwik
>            Assignee: Luke Cwik
>            Priority: Major
>              Labels: portability
>          Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> Wire up the onTimer method in the Java SDK harness FnApiDoFnRunner connecting it to
the RemoteGrpcPort read/write that is responsible for producing/consumer timers.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message