Hey all,

I bit the bullet and audited all PTransform classes in Beam Java SDK and
filed JIRA issues for all violations I could find.
I linked all them to the master JIRA issue
https://issues.apache.org/jira/browse/BEAM-1353

In general, all of these should be fixed before declaring Beam stable API.
Would appreciate if some senior folks looked at the issues and confirmed
that my suggested changes make sense.

PRs very welcome :) (though I'll be gone for the next few weeks so I can't
review right now)
Many of these are very easy to fix (a few lines of code); some require a
little more code, but as far as I can tell all of them are mechanical.


On Tue, Jan 31, 2017 at 4:10 PM Eugene Kirpichov <kirpic...@google.com>
wrote:

> On Mon, Jan 30, 2017 at 7:56 PM Dan Halperin <dhalp...@google.com.invalid>
> wrote:
>
> On Mon, Jan 30, 2017 at 5:42 PM, Eugene Kirpichov <
> kirpic...@google.com.invalid> wrote:
>
> > Hello,
> >
> > The PTransform Style Guide is live
> > https://beam.apache.org/contribute/ptransform-style-guide/ - a natural
> > next
> > step is to audit Beam libraries for compliance and file JIRAs for places
> > that need to be fixed. It'd be great to finish these cleanups before
> > declaring Beam stable API.
> >
> > Please take a look and file JIRAs / post suggestions on this thread!
> >
> > I think it'll also make a great source of easy and useful work for new
> > contributors.
> >
> > Some things I remember off the top of my head:
> > - TextIO, KafkaIO use coders improperly - coders should not be used as a
> > general-purpose byte parsing mechanism.
> >
>
> Can you say more about Kafka? Kafka actually exports byte[] by default,
> whereas Text files are String by default. So it does not seem nearly as
> egregious for Kafka as it is for Text.
>
> Agreed that KafkaIO is less egregious, but it still has methods
> withKeyCoder and withValueCoder - these should be replaced with something
> that doesn't take Coder.
>
>
>
> - HadoopFileSource is not packaged as a PTransform
> > - Some connectors, e.g. KafkaIO, should use AutoValue for their parameter
> > builders, but don't
> >
>
> Isn't AutoValue entirely an internal implementation detail that is not
> exposed(*) to users? I think this is irrelevant to a stable API.
>
> Agreed - doesn't block stable API, but still a good thing to do because it
> makes the code cleaner (for KafkaIO there's a long-standing PR that was
> blocked on ratifying the style guide
> https://github.com/apache/beam/pull/1048)
>
>
>
> (*) except that it makes transforms not able to be final, which is a
> regression.
>
> I think AutoValue use should generally be considered *very* optional. In
> transforms I author, I prefer not to use AutoValue because it makes the
> code more complex and less readable.
>
> Yeah, guidance on when to use / not use AutoValue could be improved. I
> think it makes a lot of sense when the transform has more than one or two
> parameters or when the set of parameters can grow.
>
>
>
>
> > - A few connectors improperly use
> > - Some transforms expose their transform type as "Something.Bound" and
> > "Something.Unbound", e.g. TextIO.Read.Bound - such names are banned
> >
>
> "banned" is a strong word to use here. All of these are just
> recommendations.
>
> In general yes; the goal of the style guide is to be the default, where if
> you deviate from it, you should have a good reason. I don't think there
> ever exists a good reason to name a transform Something.Bound/Unbound
> though.
>
>
>
>
> >
> > I filed an umbrella JIRA https://issues.apache.org/jira/browse/BEAM-1353
> > about
> > making existing Beam transforms comply with the guide - let's crowdsource
> > this!
> >
> > Thanks.
> >
>
>

Reply via email to