You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Reuven Lax <re...@google.com> on 2018/06/04 14:06:24 UTC

Some extensions to the DoFn API

Beam was created with an annotation-based processing API, that allows the
framework to automatically inject parameters to a DoFn's process method
(and also allows the user to mark any method as the process method
using @ProcessElement). However, these annotations were never completed. A
specific set of parameters could be injected (e.g. the window or
PipelineOptions), but for anything else you had to access it through the
ProcessContext. This limited the readability advantage of this API.

A couple of months ago I spent a bit of time extending the set of
annotations allowed. In particular, the most common uses of ProcessContext
were accessing the input element and outputting elements, and both of those
can now be done without ProcessContext. Example usage:

new DoFn<InputT, OutputT>() {
  @ProcessElement process(@Element InputT element, OutputReceiver<OutputT>
out) {
    out.output(convertInputToOutput(element));
  }
}

No need for ProcessContext anywhere in this DoFn! The Beam framework also
does type checking - if the @Element type was not InputT, you would have
seen an error. Multi-output DoFns also work, using a MultiOutputReceiver
interface.

I'll update the Beam docs later with this information, but most information
accessible from ProcessContext, OnTimerContext, StartBundleContext, or
FinishBundleContext can now be accessed via this sort of injection. The
main exceptions are side inputs and output from finishbundle, both of which
still require the context objects; however I hope to find time to provide
direct access to those as well.

pr/5331 (in progress) converts most of Beam's built-in transforms to use
this clearer style.

Reuven

Re: Some extensions to the DoFn API

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Thanks ! I will work on this one then ;)

Regards
JB

On 04/06/2018 16:55, Reuven Lax wrote:
> I'll file a JIRA to track the idea.
> 
> On Mon, Jun 4, 2018 at 5:52 PM Jean-Baptiste Onofré <jb@nanthrax.net
> <ma...@nanthrax.net>> wrote:
> 
>     Exactly, that's why something like @xpath or @json-path could be
>     interesting.
> 
>     Regards
>     JB
> 
>     On 04/06/2018 16:48, Reuven Lax wrote:
>     > Interesting. And given that Beam Schemas are recursive (a row can
>     > contain nested rows), we might actually need something like xpath
>     if we
>     > want to make this fully general.
>     >
>     > Reuven
>     >
>     > On Mon, Jun 4, 2018 at 5:45 PM Jean-Baptiste Onofré
>     <jb@nanthrax.net <ma...@nanthrax.net>
>     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> wrote:
>     >
>     >     Yup, it makes sense, it's what I had in mind.
>     >
>     >     In Apache Camel, in a Processor (similar to a DoFn), we can
>     also pass
>     >     directly languages to the arguments.
>     >
>     >     We can imagine something like:
>     >
>     >     @ProcessElement void process(@json-path("foo") String foo)
>     >
>     >     @ProcessElement void process(@xpath("//foo") String foo)
>     >
>     >     or even a expression language (simple/groovy/whatever).
>     >
>     >     Regards
>     >     JB
>     >
>     >     On 04/06/2018 16:39, Reuven Lax wrote:
>     >     > In the schema branch I have already added some annotations
>     for Schema.
>     >     > However in the future I think we could go even further and
>     allow users
>     >     > to pick individual fields out of the row schema. e.g. the
>     user might
>     >     > have a Schema with 100 fields, but only want to process
>     userId and geo
>     >     > location. I could imagine something like this
>     >     >
>     >     > @ProcessElement void process(@Field("userId") String
>     >     > userId, @Field("latitude") double lat, @Field("longitude")
>     double
>     >     long) {
>     >     > }
>     >     >
>     >     > And Beam could automatically extract the right fields for
>     the user. In
>     >     > fact we could do the same thing with KVs today - supplying
>     annotations
>     >     > to automatically unpack the KV.
>     >     >
>     >     > I do think there are a few nice ways to do side inputs as well,
>     >     but it's
>     >     > more work to design implement which is why I left it off (and
>     >     given that
>     >     > there is some design work, side input annotations should be
>     >     discussed on
>     >     > the dev list before implementation IMO).
>     >     >
>     >     > Reuven
>     >     >
>     >     > On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré
>     >     <jb@nanthrax.net <ma...@nanthrax.net>
>     <mailto:jb@nanthrax.net <ma...@nanthrax.net>>
>     >     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>
>     <mailto:jb@nanthrax.net <ma...@nanthrax.net>>>> wrote:
>     >     >
>     >     >     Hi Reuven,
>     >     >
>     >     >     That's a great improvement for user.
>     >     >
>     >     >     I don't see an easy way to have annotation about side
>     >     input/output.
>     >     >     I think we can also plan some extension annotation about
>     >     schema. Like
>     >     >     @Element(schema = foo) in addition of the type. Thoughts ?
>     >     >
>     >     >     Regards
>     >     >     JB
>     >     >
>     >     >     On 04/06/2018 16:06, Reuven Lax wrote:
>     >     >     > Beam was created with an annotation-based processing API,
>     >     that allows
>     >     >     > the framework to automatically inject parameters to a
>     DoFn's
>     >     process
>     >     >     > method (and also allows the user to mark any method as the
>     >     process
>     >     >     > method using @ProcessElement). However, these annotations
>     >     were never
>     >     >     > completed. A specific set of parameters could be injected
>     >     (e.g. the
>     >     >     > window or PipelineOptions), but for anything else you
>     had to
>     >     access it
>     >     >     > through the ProcessContext. This limited the readability
>     >     advantage of
>     >     >     > this API.
>     >     >     >
>     >     >     > A couple of months ago I spent a bit of time extending the
>     >     set of
>     >     >     > annotations allowed. In particular, the most common
>     uses of
>     >     >     > ProcessContext were accessing the input element and
>     outputting
>     >     >     elements,
>     >     >     > and both of those can now be done without ProcessContext.
>     >     Example
>     >     >     usage:
>     >     >     >
>     >     >     > new DoFn<InputT, OutputT>() {
>     >     >     >   @ProcessElement process(@Element InputT element,
>     >     >     > OutputReceiver<OutputT> out) {
>     >     >     >     out.output(convertInputToOutput(element));
>     >     >     >   }
>     >     >     > }
>     >     >     >
>     >     >     > No need for ProcessContext anywhere in this DoFn! The Beam
>     >     framework
>     >     >     > also does type checking - if the @Element type was not
>     >     InputT, you
>     >     >     would
>     >     >     > have seen an error. Multi-output DoFns also work, using a
>     >     >     > MultiOutputReceiver interface.
>     >     >     >
>     >     >     > I'll update the Beam docs later with this information,
>     but most
>     >     >     > information accessible from ProcessContext,
>     OnTimerContext,
>     >     >     > StartBundleContext, or FinishBundleContext can now be
>     >     accessed via
>     >     >     this
>     >     >     > sort of injection. The main exceptions are side inputs and
>     >     output from
>     >     >     > finishbundle, both of which still require the context
>     objects;
>     >     >     however I
>     >     >     > hope to find time to provide direct access to those as
>     well.
>     >     >     >
>     >     >     > pr/5331 (in progress) converts most of Beam's built-in
>     >     transforms
>     >     >     to use
>     >     >     > this clearer style.
>     >     >     >
>     >     >     > Reuven
>     >     >
>     >     >     --
>     >     >     Jean-Baptiste Onofré
>     >     >     jbonofre@apache.org <ma...@apache.org>
>     <mailto:jbonofre@apache.org <ma...@apache.org>>
>     >     <mailto:jbonofre@apache.org <ma...@apache.org>
>     <mailto:jbonofre@apache.org <ma...@apache.org>>>
>     >     >     http://blog.nanthrax.net
>     >     >     Talend - http://www.talend.com
>     >     >
>     >
>     >     --
>     >     Jean-Baptiste Onofré
>     >     jbonofre@apache.org <ma...@apache.org>
>     <mailto:jbonofre@apache.org <ma...@apache.org>>
>     >     http://blog.nanthrax.net
>     >     Talend - http://www.talend.com
>     >
> 
>     -- 
>     Jean-Baptiste Onofré
>     jbonofre@apache.org <ma...@apache.org>
>     http://blog.nanthrax.net
>     Talend - http://www.talend.com
> 

-- 
Jean-Baptiste Onofré
jbonofre@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com

Re: Some extensions to the DoFn API

Posted by Reuven Lax <re...@google.com>.
I'll file a JIRA to track the idea.

On Mon, Jun 4, 2018 at 5:52 PM Jean-Baptiste Onofré <jb...@nanthrax.net> wrote:

> Exactly, that's why something like @xpath or @json-path could be
> interesting.
>
> Regards
> JB
>
> On 04/06/2018 16:48, Reuven Lax wrote:
> > Interesting. And given that Beam Schemas are recursive (a row can
> > contain nested rows), we might actually need something like xpath if we
> > want to make this fully general.
> >
> > Reuven
> >
> > On Mon, Jun 4, 2018 at 5:45 PM Jean-Baptiste Onofré <jb@nanthrax.net
> > <ma...@nanthrax.net>> wrote:
> >
> >     Yup, it makes sense, it's what I had in mind.
> >
> >     In Apache Camel, in a Processor (similar to a DoFn), we can also pass
> >     directly languages to the arguments.
> >
> >     We can imagine something like:
> >
> >     @ProcessElement void process(@json-path("foo") String foo)
> >
> >     @ProcessElement void process(@xpath("//foo") String foo)
> >
> >     or even a expression language (simple/groovy/whatever).
> >
> >     Regards
> >     JB
> >
> >     On 04/06/2018 16:39, Reuven Lax wrote:
> >     > In the schema branch I have already added some annotations for
> Schema.
> >     > However in the future I think we could go even further and allow
> users
> >     > to pick individual fields out of the row schema. e.g. the user
> might
> >     > have a Schema with 100 fields, but only want to process userId and
> geo
> >     > location. I could imagine something like this
> >     >
> >     > @ProcessElement void process(@Field("userId") String
> >     > userId, @Field("latitude") double lat, @Field("longitude") double
> >     long) {
> >     > }
> >     >
> >     > And Beam could automatically extract the right fields for the
> user. In
> >     > fact we could do the same thing with KVs today - supplying
> annotations
> >     > to automatically unpack the KV.
> >     >
> >     > I do think there are a few nice ways to do side inputs as well,
> >     but it's
> >     > more work to design implement which is why I left it off (and
> >     given that
> >     > there is some design work, side input annotations should be
> >     discussed on
> >     > the dev list before implementation IMO).
> >     >
> >     > Reuven
> >     >
> >     > On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré
> >     <jb@nanthrax.net <ma...@nanthrax.net>
> >     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> wrote:
> >     >
> >     >     Hi Reuven,
> >     >
> >     >     That's a great improvement for user.
> >     >
> >     >     I don't see an easy way to have annotation about side
> >     input/output.
> >     >     I think we can also plan some extension annotation about
> >     schema. Like
> >     >     @Element(schema = foo) in addition of the type. Thoughts ?
> >     >
> >     >     Regards
> >     >     JB
> >     >
> >     >     On 04/06/2018 16:06, Reuven Lax wrote:
> >     >     > Beam was created with an annotation-based processing API,
> >     that allows
> >     >     > the framework to automatically inject parameters to a DoFn's
> >     process
> >     >     > method (and also allows the user to mark any method as the
> >     process
> >     >     > method using @ProcessElement). However, these annotations
> >     were never
> >     >     > completed. A specific set of parameters could be injected
> >     (e.g. the
> >     >     > window or PipelineOptions), but for anything else you had to
> >     access it
> >     >     > through the ProcessContext. This limited the readability
> >     advantage of
> >     >     > this API.
> >     >     >
> >     >     > A couple of months ago I spent a bit of time extending the
> >     set of
> >     >     > annotations allowed. In particular, the most common uses of
> >     >     > ProcessContext were accessing the input element and
> outputting
> >     >     elements,
> >     >     > and both of those can now be done without ProcessContext.
> >     Example
> >     >     usage:
> >     >     >
> >     >     > new DoFn<InputT, OutputT>() {
> >     >     >   @ProcessElement process(@Element InputT element,
> >     >     > OutputReceiver<OutputT> out) {
> >     >     >     out.output(convertInputToOutput(element));
> >     >     >   }
> >     >     > }
> >     >     >
> >     >     > No need for ProcessContext anywhere in this DoFn! The Beam
> >     framework
> >     >     > also does type checking - if the @Element type was not
> >     InputT, you
> >     >     would
> >     >     > have seen an error. Multi-output DoFns also work, using a
> >     >     > MultiOutputReceiver interface.
> >     >     >
> >     >     > I'll update the Beam docs later with this information, but
> most
> >     >     > information accessible from ProcessContext, OnTimerContext,
> >     >     > StartBundleContext, or FinishBundleContext can now be
> >     accessed via
> >     >     this
> >     >     > sort of injection. The main exceptions are side inputs and
> >     output from
> >     >     > finishbundle, both of which still require the context
> objects;
> >     >     however I
> >     >     > hope to find time to provide direct access to those as well.
> >     >     >
> >     >     > pr/5331 (in progress) converts most of Beam's built-in
> >     transforms
> >     >     to use
> >     >     > this clearer style.
> >     >     >
> >     >     > Reuven
> >     >
> >     >     --
> >     >     Jean-Baptiste Onofré
> >     >     jbonofre@apache.org <ma...@apache.org>
> >     <mailto:jbonofre@apache.org <ma...@apache.org>>
> >     >     http://blog.nanthrax.net
> >     >     Talend - http://www.talend.com
> >     >
> >
> >     --
> >     Jean-Baptiste Onofré
> >     jbonofre@apache.org <ma...@apache.org>
> >     http://blog.nanthrax.net
> >     Talend - http://www.talend.com
> >
>
> --
> Jean-Baptiste Onofré
> jbonofre@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>

Re: Some extensions to the DoFn API

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Exactly, that's why something like @xpath or @json-path could be
interesting.

Regards
JB

On 04/06/2018 16:48, Reuven Lax wrote:
> Interesting. And given that Beam Schemas are recursive (a row can
> contain nested rows), we might actually need something like xpath if we
> want to make this fully general.
> 
> Reuven
> 
> On Mon, Jun 4, 2018 at 5:45 PM Jean-Baptiste Onofré <jb@nanthrax.net
> <ma...@nanthrax.net>> wrote:
> 
>     Yup, it makes sense, it's what I had in mind.
> 
>     In Apache Camel, in a Processor (similar to a DoFn), we can also pass
>     directly languages to the arguments.
> 
>     We can imagine something like:
> 
>     @ProcessElement void process(@json-path("foo") String foo)
> 
>     @ProcessElement void process(@xpath("//foo") String foo)
> 
>     or even a expression language (simple/groovy/whatever).
> 
>     Regards
>     JB
> 
>     On 04/06/2018 16:39, Reuven Lax wrote:
>     > In the schema branch I have already added some annotations for Schema.
>     > However in the future I think we could go even further and allow users
>     > to pick individual fields out of the row schema. e.g. the user might
>     > have a Schema with 100 fields, but only want to process userId and geo
>     > location. I could imagine something like this
>     >
>     > @ProcessElement void process(@Field("userId") String
>     > userId, @Field("latitude") double lat, @Field("longitude") double
>     long) {
>     > }
>     >
>     > And Beam could automatically extract the right fields for the user. In
>     > fact we could do the same thing with KVs today - supplying annotations
>     > to automatically unpack the KV.
>     >
>     > I do think there are a few nice ways to do side inputs as well,
>     but it's
>     > more work to design implement which is why I left it off (and
>     given that
>     > there is some design work, side input annotations should be
>     discussed on
>     > the dev list before implementation IMO).
>     >
>     > Reuven
>     >
>     > On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré
>     <jb@nanthrax.net <ma...@nanthrax.net>
>     > <mailto:jb@nanthrax.net <ma...@nanthrax.net>>> wrote:
>     >
>     >     Hi Reuven,
>     >
>     >     That's a great improvement for user.
>     >
>     >     I don't see an easy way to have annotation about side
>     input/output.
>     >     I think we can also plan some extension annotation about
>     schema. Like
>     >     @Element(schema = foo) in addition of the type. Thoughts ?
>     >
>     >     Regards
>     >     JB
>     >
>     >     On 04/06/2018 16:06, Reuven Lax wrote:
>     >     > Beam was created with an annotation-based processing API,
>     that allows
>     >     > the framework to automatically inject parameters to a DoFn's
>     process
>     >     > method (and also allows the user to mark any method as the
>     process
>     >     > method using @ProcessElement). However, these annotations
>     were never
>     >     > completed. A specific set of parameters could be injected
>     (e.g. the
>     >     > window or PipelineOptions), but for anything else you had to
>     access it
>     >     > through the ProcessContext. This limited the readability
>     advantage of
>     >     > this API.
>     >     >
>     >     > A couple of months ago I spent a bit of time extending the
>     set of
>     >     > annotations allowed. In particular, the most common uses of
>     >     > ProcessContext were accessing the input element and outputting
>     >     elements,
>     >     > and both of those can now be done without ProcessContext.
>     Example
>     >     usage:
>     >     >
>     >     > new DoFn<InputT, OutputT>() {
>     >     >   @ProcessElement process(@Element InputT element,
>     >     > OutputReceiver<OutputT> out) {
>     >     >     out.output(convertInputToOutput(element));
>     >     >   }
>     >     > }
>     >     >
>     >     > No need for ProcessContext anywhere in this DoFn! The Beam
>     framework
>     >     > also does type checking - if the @Element type was not
>     InputT, you
>     >     would
>     >     > have seen an error. Multi-output DoFns also work, using a
>     >     > MultiOutputReceiver interface.
>     >     >
>     >     > I'll update the Beam docs later with this information, but most
>     >     > information accessible from ProcessContext, OnTimerContext,
>     >     > StartBundleContext, or FinishBundleContext can now be
>     accessed via
>     >     this
>     >     > sort of injection. The main exceptions are side inputs and
>     output from
>     >     > finishbundle, both of which still require the context objects;
>     >     however I
>     >     > hope to find time to provide direct access to those as well.
>     >     >
>     >     > pr/5331 (in progress) converts most of Beam's built-in
>     transforms
>     >     to use
>     >     > this clearer style.
>     >     >
>     >     > Reuven
>     >
>     >     --
>     >     Jean-Baptiste Onofré
>     >     jbonofre@apache.org <ma...@apache.org>
>     <mailto:jbonofre@apache.org <ma...@apache.org>>
>     >     http://blog.nanthrax.net
>     >     Talend - http://www.talend.com
>     >
> 
>     -- 
>     Jean-Baptiste Onofré
>     jbonofre@apache.org <ma...@apache.org>
>     http://blog.nanthrax.net
>     Talend - http://www.talend.com
> 

-- 
Jean-Baptiste Onofré
jbonofre@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com

Re: Some extensions to the DoFn API

Posted by Reuven Lax <re...@google.com>.
Interesting. And given that Beam Schemas are recursive (a row can contain
nested rows), we might actually need something like xpath if we want to
make this fully general.

Reuven

On Mon, Jun 4, 2018 at 5:45 PM Jean-Baptiste Onofré <jb...@nanthrax.net> wrote:

> Yup, it makes sense, it's what I had in mind.
>
> In Apache Camel, in a Processor (similar to a DoFn), we can also pass
> directly languages to the arguments.
>
> We can imagine something like:
>
> @ProcessElement void process(@json-path("foo") String foo)
>
> @ProcessElement void process(@xpath("//foo") String foo)
>
> or even a expression language (simple/groovy/whatever).
>
> Regards
> JB
>
> On 04/06/2018 16:39, Reuven Lax wrote:
> > In the schema branch I have already added some annotations for Schema.
> > However in the future I think we could go even further and allow users
> > to pick individual fields out of the row schema. e.g. the user might
> > have a Schema with 100 fields, but only want to process userId and geo
> > location. I could imagine something like this
> >
> > @ProcessElement void process(@Field("userId") String
> > userId, @Field("latitude") double lat, @Field("longitude") double long) {
> > }
> >
> > And Beam could automatically extract the right fields for the user. In
> > fact we could do the same thing with KVs today - supplying annotations
> > to automatically unpack the KV.
> >
> > I do think there are a few nice ways to do side inputs as well, but it's
> > more work to design implement which is why I left it off (and given that
> > there is some design work, side input annotations should be discussed on
> > the dev list before implementation IMO).
> >
> > Reuven
> >
> > On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré <jb@nanthrax.net
> > <ma...@nanthrax.net>> wrote:
> >
> >     Hi Reuven,
> >
> >     That's a great improvement for user.
> >
> >     I don't see an easy way to have annotation about side input/output.
> >     I think we can also plan some extension annotation about schema. Like
> >     @Element(schema = foo) in addition of the type. Thoughts ?
> >
> >     Regards
> >     JB
> >
> >     On 04/06/2018 16:06, Reuven Lax wrote:
> >     > Beam was created with an annotation-based processing API, that
> allows
> >     > the framework to automatically inject parameters to a DoFn's
> process
> >     > method (and also allows the user to mark any method as the process
> >     > method using @ProcessElement). However, these annotations were
> never
> >     > completed. A specific set of parameters could be injected (e.g. the
> >     > window or PipelineOptions), but for anything else you had to
> access it
> >     > through the ProcessContext. This limited the readability advantage
> of
> >     > this API.
> >     >
> >     > A couple of months ago I spent a bit of time extending the set of
> >     > annotations allowed. In particular, the most common uses of
> >     > ProcessContext were accessing the input element and outputting
> >     elements,
> >     > and both of those can now be done without ProcessContext. Example
> >     usage:
> >     >
> >     > new DoFn<InputT, OutputT>() {
> >     >   @ProcessElement process(@Element InputT element,
> >     > OutputReceiver<OutputT> out) {
> >     >     out.output(convertInputToOutput(element));
> >     >   }
> >     > }
> >     >
> >     > No need for ProcessContext anywhere in this DoFn! The Beam
> framework
> >     > also does type checking - if the @Element type was not InputT, you
> >     would
> >     > have seen an error. Multi-output DoFns also work, using a
> >     > MultiOutputReceiver interface.
> >     >
> >     > I'll update the Beam docs later with this information, but most
> >     > information accessible from ProcessContext, OnTimerContext,
> >     > StartBundleContext, or FinishBundleContext can now be accessed via
> >     this
> >     > sort of injection. The main exceptions are side inputs and output
> from
> >     > finishbundle, both of which still require the context objects;
> >     however I
> >     > hope to find time to provide direct access to those as well.
> >     >
> >     > pr/5331 (in progress) converts most of Beam's built-in transforms
> >     to use
> >     > this clearer style.
> >     >
> >     > Reuven
> >
> >     --
> >     Jean-Baptiste Onofré
> >     jbonofre@apache.org <ma...@apache.org>
> >     http://blog.nanthrax.net
> >     Talend - http://www.talend.com
> >
>
> --
> Jean-Baptiste Onofré
> jbonofre@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>

Re: Some extensions to the DoFn API

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Yup, it makes sense, it's what I had in mind.

In Apache Camel, in a Processor (similar to a DoFn), we can also pass
directly languages to the arguments.

We can imagine something like:

@ProcessElement void process(@json-path("foo") String foo)

@ProcessElement void process(@xpath("//foo") String foo)

or even a expression language (simple/groovy/whatever).

Regards
JB

On 04/06/2018 16:39, Reuven Lax wrote:
> In the schema branch I have already added some annotations for Schema.
> However in the future I think we could go even further and allow users
> to pick individual fields out of the row schema. e.g. the user might
> have a Schema with 100 fields, but only want to process userId and geo
> location. I could imagine something like this
> 
> @ProcessElement void process(@Field("userId") String
> userId, @Field("latitude") double lat, @Field("longitude") double long) {
> }
> 
> And Beam could automatically extract the right fields for the user. In
> fact we could do the same thing with KVs today - supplying annotations
> to automatically unpack the KV.
> 
> I do think there are a few nice ways to do side inputs as well, but it's
> more work to design implement which is why I left it off (and given that
> there is some design work, side input annotations should be discussed on
> the dev list before implementation IMO).
> 
> Reuven
> 
> On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré <jb@nanthrax.net
> <ma...@nanthrax.net>> wrote:
> 
>     Hi Reuven,
> 
>     That's a great improvement for user.
> 
>     I don't see an easy way to have annotation about side input/output.
>     I think we can also plan some extension annotation about schema. Like
>     @Element(schema = foo) in addition of the type. Thoughts ?
> 
>     Regards
>     JB
> 
>     On 04/06/2018 16:06, Reuven Lax wrote:
>     > Beam was created with an annotation-based processing API, that allows
>     > the framework to automatically inject parameters to a DoFn's process
>     > method (and also allows the user to mark any method as the process
>     > method using @ProcessElement). However, these annotations were never
>     > completed. A specific set of parameters could be injected (e.g. the
>     > window or PipelineOptions), but for anything else you had to access it
>     > through the ProcessContext. This limited the readability advantage of
>     > this API.
>     >
>     > A couple of months ago I spent a bit of time extending the set of
>     > annotations allowed. In particular, the most common uses of
>     > ProcessContext were accessing the input element and outputting
>     elements,
>     > and both of those can now be done without ProcessContext. Example
>     usage:
>     >
>     > new DoFn<InputT, OutputT>() {
>     >   @ProcessElement process(@Element InputT element,
>     > OutputReceiver<OutputT> out) {
>     >     out.output(convertInputToOutput(element));
>     >   }
>     > }
>     >
>     > No need for ProcessContext anywhere in this DoFn! The Beam framework
>     > also does type checking - if the @Element type was not InputT, you
>     would
>     > have seen an error. Multi-output DoFns also work, using a
>     > MultiOutputReceiver interface.
>     >
>     > I'll update the Beam docs later with this information, but most
>     > information accessible from ProcessContext, OnTimerContext,
>     > StartBundleContext, or FinishBundleContext can now be accessed via
>     this
>     > sort of injection. The main exceptions are side inputs and output from
>     > finishbundle, both of which still require the context objects;
>     however I
>     > hope to find time to provide direct access to those as well.
>     >
>     > pr/5331 (in progress) converts most of Beam's built-in transforms
>     to use
>     > this clearer style.
>     >
>     > Reuven
> 
>     -- 
>     Jean-Baptiste Onofré
>     jbonofre@apache.org <ma...@apache.org>
>     http://blog.nanthrax.net
>     Talend - http://www.talend.com
> 

-- 
Jean-Baptiste Onofré
jbonofre@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com

Re: Some extensions to the DoFn API

Posted by Reuven Lax <re...@google.com>.
In the schema branch I have already added some annotations for Schema.
However in the future I think we could go even further and allow users to
pick individual fields out of the row schema. e.g. the user might have a
Schema with 100 fields, but only want to process userId and geo location. I
could imagine something like this

@ProcessElement void process(@Field("userId") String
userId, @Field("latitude") double lat, @Field("longitude") double long) {
}

And Beam could automatically extract the right fields for the user. In fact
we could do the same thing with KVs today - supplying annotations to
automatically unpack the KV.

I do think there are a few nice ways to do side inputs as well, but it's
more work to design implement which is why I left it off (and given that
there is some design work, side input annotations should be discussed on
the dev list before implementation IMO).

Reuven

On Mon, Jun 4, 2018 at 5:29 PM Jean-Baptiste Onofré <jb...@nanthrax.net> wrote:

> Hi Reuven,
>
> That's a great improvement for user.
>
> I don't see an easy way to have annotation about side input/output.
> I think we can also plan some extension annotation about schema. Like
> @Element(schema = foo) in addition of the type. Thoughts ?
>
> Regards
> JB
>
> On 04/06/2018 16:06, Reuven Lax wrote:
> > Beam was created with an annotation-based processing API, that allows
> > the framework to automatically inject parameters to a DoFn's process
> > method (and also allows the user to mark any method as the process
> > method using @ProcessElement). However, these annotations were never
> > completed. A specific set of parameters could be injected (e.g. the
> > window or PipelineOptions), but for anything else you had to access it
> > through the ProcessContext. This limited the readability advantage of
> > this API.
> >
> > A couple of months ago I spent a bit of time extending the set of
> > annotations allowed. In particular, the most common uses of
> > ProcessContext were accessing the input element and outputting elements,
> > and both of those can now be done without ProcessContext. Example usage:
> >
> > new DoFn<InputT, OutputT>() {
> >   @ProcessElement process(@Element InputT element,
> > OutputReceiver<OutputT> out) {
> >     out.output(convertInputToOutput(element));
> >   }
> > }
> >
> > No need for ProcessContext anywhere in this DoFn! The Beam framework
> > also does type checking - if the @Element type was not InputT, you would
> > have seen an error. Multi-output DoFns also work, using a
> > MultiOutputReceiver interface.
> >
> > I'll update the Beam docs later with this information, but most
> > information accessible from ProcessContext, OnTimerContext,
> > StartBundleContext, or FinishBundleContext can now be accessed via this
> > sort of injection. The main exceptions are side inputs and output from
> > finishbundle, both of which still require the context objects; however I
> > hope to find time to provide direct access to those as well.
> >
> > pr/5331 (in progress) converts most of Beam's built-in transforms to use
> > this clearer style.
> >
> > Reuven
>
> --
> Jean-Baptiste Onofré
> jbonofre@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>

Re: Some extensions to the DoFn API

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Hi Reuven,

That's a great improvement for user.

I don't see an easy way to have annotation about side input/output.
I think we can also plan some extension annotation about schema. Like
@Element(schema = foo) in addition of the type. Thoughts ?

Regards
JB

On 04/06/2018 16:06, Reuven Lax wrote:
> Beam was created with an annotation-based processing API, that allows
> the framework to automatically inject parameters to a DoFn's process
> method (and also allows the user to mark any method as the process
> method using @ProcessElement). However, these annotations were never
> completed. A specific set of parameters could be injected (e.g. the
> window or PipelineOptions), but for anything else you had to access it
> through the ProcessContext. This limited the readability advantage of
> this API.
> 
> A couple of months ago I spent a bit of time extending the set of
> annotations allowed. In particular, the most common uses of
> ProcessContext were accessing the input element and outputting elements,
> and both of those can now be done without ProcessContext. Example usage:
> 
> new DoFn<InputT, OutputT>() {
>   @ProcessElement process(@Element InputT element,
> OutputReceiver<OutputT> out) {
>     out.output(convertInputToOutput(element));
>   }
> }
> 
> No need for ProcessContext anywhere in this DoFn! The Beam framework
> also does type checking - if the @Element type was not InputT, you would
> have seen an error. Multi-output DoFns also work, using a
> MultiOutputReceiver interface.
> 
> I'll update the Beam docs later with this information, but most
> information accessible from ProcessContext, OnTimerContext,
> StartBundleContext, or FinishBundleContext can now be accessed via this
> sort of injection. The main exceptions are side inputs and output from
> finishbundle, both of which still require the context objects; however I
> hope to find time to provide direct access to those as well.
> 
> pr/5331 (in progress) converts most of Beam's built-in transforms to use
> this clearer style.
> 
> Reuven

-- 
Jean-Baptiste Onofré
jbonofre@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com