Add DoFn.OnTimerContext
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/3f8c8076 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/3f8c8076 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/3f8c8076 Branch: refs/heads/master Commit: 3f8c80769a3bb38da64c6628fd8c4669fcac794b Parents: 6807480 Author: Kenneth Knowles <k...@google.com> Authored: Tue Dec 6 20:10:06 2016 -0800 Committer: Kenneth Knowles <k...@google.com> Committed: Wed Dec 7 19:22:43 2016 -0800 ---------------------------------------------------------------------- .../org/apache/beam/sdk/transforms/DoFn.java | 22 ++++++++++++++++++++ 1 file changed, 22 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f8c8076/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 7aabec9..699403f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.state.State; @@ -295,6 +296,27 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD } /** + * Information accessible when running a {@link DoFn.OnTimer} method. + */ + public abstract class OnTimerContext extends Context { + + /** + * Returns the timestamp of the current timer. + */ + public abstract Instant timestamp(); + + /** + * Returns the window in which the timer is firing. + */ + public abstract BoundedWindow window(); + + /** + * Returns the time domain of the current timer. + */ + public abstract TimeDomain timeDomain(); + } + + /** * Returns the allowed timestamp skew duration, which is the maximum * duration that timestamps can be shifted backward in * {@link DoFn.Context#outputWithTimestamp}.