You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "Kenneth Knowles (JIRA)" <ji...@apache.org> on 2017/01/19 21:12:26 UTC

[jira] [Updated] (BEAM-1283) DoFn.Context.output spec for startBundle/finishBundle is a mess

     [ https://issues.apache.org/jira/browse/BEAM-1283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Kenneth Knowles updated BEAM-1283:
----------------------------------
    Description: 
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.

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

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.


> DoFn.Context.output spec for startBundle/finishBundle is a mess
> ---------------------------------------------------------------
>
>                 Key: BEAM-1283
>                 URL: https://issues.apache.org/jira/browse/BEAM-1283
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model, sdk-java-core
>            Reporter: Kenneth Knowles
>            Assignee: Kenneth Knowles
>
> 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.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)