+1

At first I liked the API but was skeptical because I though that this would
require reflective invocation. Then I read on and saw that code generation
is used and was convinced. :-)

I especially like how it both cleans up the API and allows more
optimizations in the future, especially with side inputs and the different
methods for emitting.

On Wed, 27 Jul 2016 at 06:49 Jean-Baptiste Onofré <[email protected]> wrote:

>
>
> +1
> I like the proposal and great description.
> ThanksRegards JB
>
> -------- Original message --------
> From: Kenneth Knowles <[email protected]>
> Date: 27/07/2016  05:29  (GMT+01:00)
> To: [email protected]
> Subject: [PROPOSAL] A brand new DoFn
>
> Hi all,
>
> I have a major new feature to propose: the next generation of DoFn.
>
> It sounds a bit grandiose, but I think it is the best way to understand the
> proposal.
>
> This is strongly motivated by the design for state and timers, aka "per-key
> workflows". Since the two features are separable and have separate design
> docs, I have started a separate thread for each.
>
> To get a quick overview of the proposal for a new DoFn, and how it improves
> upon the flexibility and validation of DoFn, browse this presentation:
>
>   https://s.apache.org/presenting-a-new-dofn
>
> Due to the extent of this proposal, Ben & I have also prepared an in-depth
> document at https://s.apache.org/a-new-dofn with additional details.
> Please
> comment on particulars there, or just reply to this email.
>
> The remainder of this email is yet another summary of the proposal, to
> entice you to read the documents above and respond with a "+1".
>
> This is a feature that has been an experimental feature of the Java SDK for
> some time, under the name DoFnWithContext. For the purposes of this email
> and the linked documents, I will call it NewDoFn and I will call the status
> quo OldDoFn.
>
> The differences between NewDoFn and and OldDoFn are most easily understood
> with a quick code snippet:
>
>     new OldDoFn<Foo, Baz>() {
>       @Override
>       public void processElement(ProcessContext c) { … }
>     }
>
>     new NewDoFn<Foo, Baz>() {
>       @ProcessElement   // <-- This is the only difference
>       public void processElement(ProcessContext c) { … }
>     }
>
> What changed? NewDoFn uses annotation-based dispatch instead of method
> overrides. The method annotated with @ProcessElement is used to process
> elements. It can have any name or signature, and validation is performed at
> pipeline construction time.
>
> Why do this? It allows the argument list for processElement to change. This
> approach gives NewDoFn many advantages, which are demonstrated at length in
> the linked documents. Here are some highlights:
>
>  - Simpler backwards-compatible approaches to new features
>  - Simpler composition of advanced features
>  - Greater pipeline construction-time validation
>  - Easier evolution of a simple anonymous DoFn into one that uses advanced
> features
>
> Here are some abbreviated demonstrations of things that work today or could
> work easily with NewDoFn but require complex interrelated designs without
> it:
>
> Access the element's window:
>
>     new NewDoFn<Foo, Baz>() {
>       @ProcessElement
>       public void processElement(ProcessContext c, BoundedWindow w) { … }
>     }
>
> Use persistent state:
>
>     new NewDoFn<Foo, Baz>() {
>       @ProcessElement
>       public void processElement(
>           ProcessContext c,
>           @StateId("cell-id") ValueState<MyStateValue> state) {
>         …
>       }
>     }
>
> Set and receive timers:
>
>     new NewDoFn<Foo, Baz>() {
>       @ProcessElement
>       public void processElement(
>           ProcessContext c,
>           @TimerId("timer-id") Timer state) {
>         …
>       }
>
>       @OnTimer("timer-id")
>       void onMyTimer(OnTimerContext) { … }
>     }
>
> Receive a side input as a parameter:
>
>     new NewDoFn<Foo, Baz>() {
>       @ProcessElement
>       public void processElement(
>           ProcessContext c,
>           @SideInput Supplier<T> side) {
>         …
>       }
>     }
>
> So this is what I am proposing: We should move the Beam Java SDK to
> NewDoFn!
>
> My proposed migration plan is:
>
> 1. leave a git tag before anything, so users can pin to it
> 2. mv DoFn OldDoFn && mv DoFnWithContext DoFn
> 3. get everything working with all runners
> 4. rm OldDoFn # a few weeks later
>
> This will affect bleeding edge users, who will need to replace @Override
> with @ProcessElement in all their DoFns. They can also pin to a commit
> prior to the change or temporarily replace DoFn with OldDoFn everywhere.
>
> I've already done step 2 in a branch at
> https://github.com/kennknowles/incubator-beam/DoFnWithContext and ported a
> few examples in their own commits. If you view those commits, you can see
> how simple the migration path is.
>
> Please let me know what you think. It is a big change, but one that I think
> yields pretty nice rewards.
>
> Kenn
>

Reply via email to