+1 the runner would "special-case" these timer PCollections to wire them
with its native timer management. At least in the Java/FnDataReceiver case
that looks straightforward.
Thomas
On Mon, Jun 4, 2018 at 3:46 PM, Lukasz Cwik wrote:
> Fixed the permissions, feel free to comment on the doc.
>
>
Fixed the permissions, feel free to comment on the doc.
The specs on the ParDoPayload will stay, analogous to the SideInputPayload.
The PCollection will not be modified and will continue to contain the
windowing strategy and coder.
On Mon, Jun 4, 2018 at 3:41 PM Kenneth Knowles wrote:
> I like
I like it. Having the extra portability layer really opens up these
possibilities that wouldn't make a usable API for a user, but are really
helpful for modeling.
I've only got View permissions to the doc, so commenting here. You mention
that they are modeled as a PCollection, but it seems that th
I have been working on a proposal for adding support for timers to the
Apache Beam portability APIs.
The synopsis is to model timers as PCollections. This allows us to treat
timers as just another type of data that is transmitted/received by a
Runner during execution and leverage all the work that