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

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

                Author: ASF GitHub Bot
            Created on: 08/Dec/19 10:24
            Start Date: 08/Dec/19 10:24
    Worklog Time Spent: 10m 
      Work Description: JozoVilcek commented on pull request #8774: [BEAM-8550] 
Requires time sorted input
URL: https://github.com/apache/beam/pull/8774#discussion_r355172757
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
 ##########
 @@ -656,6 +656,24 @@ public Duration getAllowedTimestampSkew() {
   @Target(ElementType.METHOD)
   public @interface RequiresStableInput {}
 
+  /**
+   * <b><i>Experimental - no backwards compatibility guarantees. The exact 
name or usage of this
+   * feature may change.</i></b>
+   *
+   * <p>Annotation that may be added to a {@link ProcessElement} method to 
indicate that the runner
+   * must ensure that the observable contents of the input {@link PCollection} 
is sorted by time, in
+   * ascending order. The time ordering is generally defined by element's 
timestamp, but an
+   * alternative user supplied ordering function can be supplied.
+   *
+   * <p>Note that this annotation makes sense only for stateful {@code 
ParDo}s, because outcome of
+   * stateless functions cannot depend on the ordering.
+   */
+  @Documented
+  @Experimental
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  public @interface RequiresTimeSortedInput {}
 
 Review comment:
   I think doc should be enriched with more details and tradeoffs of this 
annotations. Definitely impact on handling late data needs to be mentioned.
 
----------------------------------------------------------------
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: 355794)
    Time Spent: 3h 10m  (was: 3h)

> @RequiresTimeSortedInput DoFn annotation
> ----------------------------------------
>
>                 Key: BEAM-8550
>                 URL: https://issues.apache.org/jira/browse/BEAM-8550
>             Project: Beam
>          Issue Type: New Feature
>          Components: beam-model, sdk-java-core
>            Reporter: Jan Lukavský
>            Assignee: Jan Lukavský
>            Priority: Major
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> Implement new annotation {{@RequiresTimeSortedInput}} for stateful DoFn as 
> described in [design 
> document|https://docs.google.com/document/d/1ObLVUFsf1NcG8ZuIZE4aVy2RYKx2FfyMhkZYWPnI9-c/edit?usp=sharing].
>  First implementation will assume that:
>   - time is defined by timestamp in associated WindowedValue
>   - allowed lateness is explicitly zero and all late elements are dropped 
> (due to being out of order)
> The above properties are considered temporary and will be resolved by 
> subsequent extensions (backwards compatible).



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

Reply via email to