[ 
https://issues.apache.org/jira/browse/BEAM-1283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15987893#comment-15987893
 ] 

ASF GitHub Bot commented on BEAM-1283:
--------------------------------------

GitHub user sb2nov opened a pull request:

    https://github.com/apache/beam/pull/2753

    [BEAM-1283] Finish bundle should only emit windowed values

    Be sure to do all of the following to help us incorporate your contribution
    quickly and easily:
    
     - [ ] Make sure the PR title is formatted like:
       `[BEAM-<Jira issue #>] Description of pull request`
     - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable
           Travis-CI on your fork and ensure the whole test matrix passes).
     - [ ] Replace `<Jira issue #>` in the title with the actual Jira issue
           number, if there is one.
     - [ ] If this contribution is large, please file an Apache
           [Individual Contributor License 
Agreement](https://www.apache.org/licenses/icla.pdf).
    
    ---
    
    R: @aaltay @robertwb PTAL

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sb2nov/beam 
BEAM-1283-finish-bundle-windowed-values

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/beam/pull/2753.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2753
    
----
commit ace66e63eca538d7b5d4b9369cae63531247d6f5
Author: Sourabh Bajaj <[email protected]>
Date:   2017-04-27T23:11:56Z

    [BEAM-1283] Finish bundle should only emit windowed values

----


> DoFn finishBundle should be required to specify the window for output
> ---------------------------------------------------------------------
>
>                 Key: BEAM-1283
>                 URL: https://issues.apache.org/jira/browse/BEAM-1283
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model, sdk-java-core, sdk-py
>            Reporter: Kenneth Knowles
>            Assignee: Thomas Groh
>              Labels: backward-incompatible
>             Fix For: First stable release
>
>
> The spec is here in Javadoc: 
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java#L128
> "If invoked from {{@StartBundle}} or {{@FinishBundle}}, this will attempt to 
> use the {{WindowFn}} of the input {{PCollection}} to determine what windows 
> the element should be in, throwing an exception if the {{WindowFn}} attempts 
> to access any information about the input element. The output element will 
> have a timestamp of negative infinity."
> This is a collection of caveats that make this method not always technically 
> wrong, but quite a mess. Ideas that reasonable folks have suggested lately:
>  - The {{WindowFn}} cannot actually be applied because {{WindowFn}} is 
> allowed to see the element type. The spec just avoids this by limiting which 
> {{WindowFn}} can be used.
>  - There is no natural output timestamp, so it should always be provided. The 
> spec avoids this by specifying an arbitrary and fairly useless timestamp.
>  - If it is a merging {{WindowFn}} like sessions that has already been merged 
> then you'll just have a bogus proto window regardless of explicit timestamp 
> or not.
> The use cases for these methods are best addressed by state plus window 
> expiry callback, so we should revisit this spec and probably just wipe it.
> There are some rare case where you might need to output from {{FinishBundle}} 
> in a way that is not _actually_ sensitive to bundling (perhaps modulo some 
> downstream notion of equivalence) in which case you had better know what 
> window you are outputting to. Often it should be the global window.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to