One interesting wrinkle: I'm about to propose a set of semantics for
snapshotting/in-place updating pipelines. Part of this proposal is the
ability to write pipelines to "upgrade" snapshots to make them compatible
with new graphs. This relies on the ability to have two separate coders for
the same type - the old coder and the new coder - in order to handle the
case where the user has changed coders in the new pipeline.

On Tue, Aug 1, 2017 at 2:12 PM, Robert Bradshaw <rober...@google.com.invalid
> wrote:

> There are two concerns in this thread:
>
> (1) Getting rid of PCollection.setCoder(). Everyone seems in favor of this
> (right?)
>
> (2) Deprecating specifying Coders in favor of specifying TypeDescriptors.
> I'm generally in favor, but it's unclear how far we can push this through.
>
> Let's at least do (1), and separately state a preference for (2), seeing
> how fare we can push it.
>
> On Thu, Jul 27, 2017 at 9:13 PM, Kenneth Knowles <k...@google.com.invalid>
> wrote:
>
> > Another thought on this: setting a custom coder to support a special data
> > distribution is likely often a property of the input to the pipeline. So
> > setting a coder during pipeline construction - more generally, when
> writing
> > a composite transform for reuse - you might not actually have the needed
> > information. But setting up a special indicator type descriptor lets your
> > users map that type descriptor to a coder that works well for their data.
> >
> > But Robert's example of RawUnionValue seems like a deal breaker for all
> > approaches. It really requires .getCoder() during expand() and explicitly
> > building coders encoding information that is cumbersome to get into a
> > TypeDescriptor. While making up new type languages is a comfortable
> > activity for me :-) I don't think we should head down that path, for our
> > users' sake. So I'll stop hoping we can eliminate this pain point for
> now.
> >
> > Kenn
> >
> > On Thu, Jul 27, 2017 at 8:48 PM, Kenneth Knowles <k...@google.com> wrote:
> >
> > > On Thu, Jul 27, 2017 at 11:18 AM, Thomas Groh <tg...@google.com.invalid
> >
> > > wrote:
> > >
> > >> introduce a
> > >> new, specialized type to represent the restricted
> > >> (alternatively-distributed?) data. The TypeDescriptor for this type
> can
> > >> map
> > >> to the specialized coder, without having to perform a significant
> degree
> > >> of
> > >> potentially wasted encoding work, plus it includes the assumptions
> that
> > >> are
> > >> being made about the distribution of data.
> > >>
> > >
> > > This is a very cool idea, in theory :-)
> > >
> > > For complex types with a few allocations involved and/or nontrivial
> > > deserialization, or when a pipeline does a lot of real work, I think
> the
> > > wrapper cost won't be perceptible.
> > >
> > > But  for more primitive types in pipelines that don't really do much
> > > computation but just move data around, I think it could matter.
> Certainly
> > > there are languages with constructs to allow type wrappers at zero cost
> > > (Haskell's `newtype`).
> > >
> > > This is all just speculation until we measure, like most of this
> thread.
> > >
> > > Kenn
> > >
> > >
> > >> > On Thu, Jul 27, 2017 at 11:00 AM, Thomas Groh
> > <tg...@google.com.invalid
> > >> >
> > >> > wrote:
> > >> >
> > >> > > +1 on getting rid of setCoder; just from a Java SDK perspective,
> my
> > >> ideal
> > >> > > world contains PCollections which don't have a user-visible way to
> > >> mutate
> > >> > > them.
> > >> > >
> > >> > > My preference would be to use TypeDescriptors everywhere within
> > >> Pipeline
> > >> > > construction (where possible), and utilize the CoderRegistry
> > >> everywhere
> > >> > to
> > >> > > actually extract the appropriate type. The unfortunate difficulty
> of
> > >> > having
> > >> > > to encode a union type and the lack of variable-length generics
> does
> > >> > > complicate that. We could consider some way of constructing coders
> > in
> > >> the
> > >> > > registry from a collection of type descriptors (which should be
> > >> > accessible
> > >> > > from the point the union-type is being constructed), e.g.
> something
> > >> like
> > >> > > `getCoder(TypeDescriptor output, TypeDescriptor... components)` -
> > that
> > >> > does
> > >> > > only permit a single flat level (but since this is being invoked
> by
> > >> the
> > >> > SDK
> > >> > > during construction it could also pass Coder...).
> > >> > >
> > >> > >
> > >> > >
> > >> > > On Thu, Jul 27, 2017 at 10:22 AM, Robert Bradshaw <
> > >> > > rober...@google.com.invalid> wrote:
> > >> > >
> > >> > > > On Thu, Jul 27, 2017 at 10:04 AM, Kenneth Knowles
> > >> > > > <k...@google.com.invalid> wrote:
> > >> > > > > On Thu, Jul 27, 2017 at 2:22 AM, Lukasz Cwik
> > >> > <lc...@google.com.invalid
> > >> > > >
> > >> > > > > wrote:
> > >> > > > >>
> > >> > > > >> Ken/Robert, I believe users will want the ability to set the
> > >> output
> > >> > > > coder
> > >> > > > >> because coders may have intrinsic properties where the type
> > isn't
> > >> > > enough
> > >> > > > >> information to fully specify what I want as a user. Some
> cases
> > I
> > >> can
> > >> > > see
> > >> > > > >> are:
> > >> > > > >> 1) I have a cheap and fast non-deterministic coder but a
> > >> different
> > >> > > > slower
> > >> > > > >> coder when I want to use it as the key to a GBK, For example
> > >> with a
> > >> > > set
> > >> > > > >> coder, it would need to consistently order the values of the
> > set
> > >> > when
> > >> > > > used
> > >> > > > >> as the key.
> > >> > > > >> 2) I know a property of the data which allows me to have a
> > >> cheaper
> > >> > > > >> encoding. Imagine I know that all the strings have a common
> > >> prefix
> > >> > or
> > >> > > > >> integers that are in a certain range, or that a matrix is
> > >> > > sparse/dense.
> > >> > > > Not
> > >> > > > >> all PCollections of strings / integers / matrices in the
> > pipeline
> > >> > will
> > >> > > > have
> > >> > > > >> this property, just some.
> > >> > > > >> 3) Sorting comes up occasionally, traditionally in Google
> this
> > >> was
> > >> > > done
> > >> > > > by
> > >> > > > >> sorting the encoded version of the object lexicographically
> > >> during a
> > >> > > > GBK.
> > >> > > > >> There are good lexicographical byte representations for ASCII
> > >> > strings,
> > >> > > > >> integers, and for some IEEE number representations which
> could
> > be
> > >> > done
> > >> > > > by
> > >> > > > >> the use of a special coder.
> > >> > > > >>
> > >> > > > >
> > >> > > > > Items (1) and (3) do not require special knowledge from the
> > user.
> > >> > They
> > >> > > > are
> > >> > > > > easily observed properties of a pipeline. My proposal included
> > >> full
> > >> > > > > automation for both. The suggestion is new methods
> > >> > > > > .getDeterministicCoder(TypeDescriptor) and
> > >> > > > > .getLexicographicCoder(TypeDescriptor).
> > >> > > >
> > >> > > > Completely agree--usecases (1) and (3) are an indirect use of
> > Coders
> > >> > > > that are used to achieve an effect that would be better
> expressed
> > >> > > > directly.
> > >> > > >
> > >> > > > > (2) is an interesting hypothetical for massive scale where
> tiny
> > >> > > > incremental
> > >> > > > > optimization represents a lot of cost _and_ your data has
> > >> sufficient
> > >> > > > > structure to realize a benefit _and_ it needs to be pinpointed
> > to
> > >> > just
> > >> > > > some
> > >> > > > > PCollections. I think our experience with coders so far is
> that
> > >> their
> > >> > > > > existence is almost entirely negative. It would be nice to
> > support
> > >> > this
> > >> > > > > vanishingly rare case without inflicting a terrible pain point
> > on
> > >> the
> > >> > > > model
> > >> > > > > and all other users.
> > >> > > >
> > >> > > > (2) is not just about cheapness, sometimes there's other
> structure
> > >> in
> > >> > > > the data we can leverage. Consider the UnionCoder used in
> > >> > > > CoGBK--RawUnionValue has an integer value that specifies
> indicates
> > >> the
> > >> > > > type of it's raw Object field. Unless we want to extend the type
> > >> > > > language, there's not a sufficient type descriptor that can be
> > used
> > >> to
> > >> > > > infer the coder. I'm dubious going down the road of adding
> special
> > >> > > > cases is the right thing here.
> > >> > > >
> > >> > > > > For example, in those cases you could encode in your
> > >> > > > > DoFn so the type descriptor would just be byte[].
> > >> > > >
> > >> > > > As well as being an extremely cumbersome API, this would incur
> the
> > >> > > > cost of coding/decoding at that DoFn boundary even if it is
> fused
> > >> > > > away.
> > >> > > >
> > >> > > > >> On Thu, Jul 27, 2017 at 1:34 AM, Jean-Baptiste Onofré <
> > >> > > j...@nanthrax.net>
> > >> > > > >> wrote:
> > >> > > > >>
> > >> > > > >> > Hi,
> > >> > > > >> >
> > >> > > > >> > That's an interesting thread and I was wondering the
> > >> relationship
> > >> > > > between
> > >> > > > >> > type descriptor and coder for a while ;)
> > >> > > > >> >
> > >> > > > >> > Today, in a PCollection, we can set the coder and we also
> > have
> > >> a
> > >> > > > >> > getTypeDescriptor(). It sounds weird to me: it should be
> one
> > or
> > >> > the
> > >> > > > >> other.
> > >> > > > >> >
> > >> > > > >> > Basically, if the Coder is not used to define the type,
> > than, I
> > >> > > fully
> > >> > > > >> > agree with Eugene.
> > >> > > > >> >
> > >> > > > >> > Basically, the PCollection should define only the type
> > >> descriptor,
> > >> > > not
> > >> > > > >> the
> > >> > > > >> > coder by itself: the coder can be found using the type
> > >> descriptor.
> > >> > > > >> >
> > >> > > > >> > With both coder and type descriptor on the PCollection, it
> > >> sounds
> > >> > a
> > >> > > > big
> > >> > > > >> > "decoupled" to me and it would be possible to have a coder
> on
> > >> the
> > >> > > > >> > PCollection that doesn't match the type descriptor.
> > >> > > > >> >
> > >> > > > >> > I think PCollection type descriptor should be defined, and
> > the
> > >> > coder
> > >> > > > >> > should be implicit based on this type descriptor.
> > >> > > > >> >
> > >> > > > >> > Thoughts ?
> > >> > > > >> >
> > >> > > > >> > Regards
> > >> > > > >> > JB
> > >> > > > >> >
> > >> > > > >> >
> > >> > > > >> > On 07/26/2017 05:25 AM, Eugene Kirpichov wrote:
> > >> > > > >> >
> > >> > > > >> >> Hello,
> > >> > > > >> >>
> > >> > > > >> >> I've worked on a few different things recently and ran
> > >> repeatedly
> > >> > > > into
> > >> > > > >> the
> > >> > > > >> >> same issue: that we do not have clear guidance on who
> should
> > >> set
> > >> > > the
> > >> > > > >> Coder
> > >> > > > >> >> on a PCollection: is it responsibility of the PTransform
> > that
> > >> > > outputs
> > >> > > > >> it,
> > >> > > > >> >> or is it responsibility of the user, or is it sometimes
> one
> > >> and
> > >> > > > >> sometimes
> > >> > > > >> >> the other?
> > >> > > > >> >>
> > >> > > > >> >> I believe that the answer is "it's responsibility of the
> > >> > transform"
> > >> > > > and
> > >> > > > >> >> moreover that  ideally PCollection.setCoder() should not
> > >> exist.
> > >> > > > Instead:
> > >> > > > >> >>
> > >> > > > >> >> - Require that all transforms set a Coder on the
> > PCollection's
> > >> > they
> > >> > > > >> >> produce
> > >> > > > >> >> - i.e. it should never be responsibility of the user to
> "fix
> > >> up"
> > >> > a
> > >> > > > coder
> > >> > > > >> >> on
> > >> > > > >> >> a PCollection produced by a transform.
> > >> > > > >> >>
> > >> > > > >> >> - Since all transforms are composed of primitive
> transforms,
> > >> > saying
> > >> > > > >> >> "transforms must set a Coder" means simply that all
> > >> *primitive*
> > >> > > > >> transforms
> > >> > > > >> >> must set a Coder on their output.
> > >> > > > >> >>
> > >> > > > >> >> - In some cases, a primitive PTransform currently doesn't
> > have
> > >> > > enough
> > >> > > > >> >> information to infer a coder for its output collection -
> > e.g.
> > >> > > > >> >> ParDo.of(DoFn<InputT, OutputT>) might be unable to infer a
> > >> coder
> > >> > > for
> > >> > > > >> >> OutputT. In that case such transforms should allow the
> user
> > to
> > >> > > > provide a
> > >> > > > >> >> coder: ParDo.of(DoFn).withOutputCoder(...) [note that
> this
> > >> > differs
> > >> > > > from
> > >> > > > >> >> requiring the user to set a coder on the resulting
> > collection]
> > >> > > > >> >>
> > >> > > > >> >> - Corollary: composite transforms need to only configure
> > their
> > >> > > > primitive
> > >> > > > >> >> transforms (and composite sub-transforms) properly, and
> give
> > >> > them a
> > >> > > > >> Coder
> > >> > > > >> >> if needed.
> > >> > > > >> >>
> > >> > > > >> >> - Corollary: a PTransform with type parameters <FooT,
> BarT,
> > >> ...>
> > >> > > > needs
> > >> > > > >> to
> > >> > > > >> >> be configurable with coders for all of these, because the
> > >> > > > implementation
> > >> > > > >> >> of
> > >> > > > >> >> the transform may change and it may introduce intermediate
> > >> > > > collections
> > >> > > > >> >> involving these types. However, in many cases, some of
> these
> > >> type
> > >> > > > >> >> parameters appear in the type of the transform's input,
> > e.g. a
> > >> > > > >> >> PTransform<PCollection<KV<FooT, BarT>>,
> PCollection<MooT>>
> > >> will
> > >> > > > always
> > >> > > > >> be
> > >> > > > >> >> able to extract the coders for FooT and BarT from the
> input
> > >> > > > PCollection,
> > >> > > > >> >> so
> > >> > > > >> >> the user does not need to provide them. However, a coder
> for
> > >> BarT
> > >> > > > must
> > >> > > > >> be
> > >> > > > >> >> provided. I think in most cases the transform needs to be
> > >> > > > configurable
> > >> > > > >> >> only
> > >> > > > >> >> with coders for its output.
> > >> > > > >> >>
> > >> > > > >> >> Here's a smooth migration path to accomplish the above:
> > >> > > > >> >> - Make PCollection.createPrimitiveOutputInternal() take a
> > >> Coder.
> > >> > > > >> >> - Make all primitive transforms optionally configurable
> > with a
> > >> > > coder
> > >> > > > for
> > >> > > > >> >> their outputs, such as ParDo.of(DoFn).withOutputCoder().
> > >> > > > >> >> - By using the above, make all composite transforms
> shipped
> > >> with
> > >> > > the
> > >> > > > SDK
> > >> > > > >> >> set a Coder on the collections they produce; in some
> cases,
> > >> this
> > >> > > will
> > >> > > > >> >> require adding a withSomethingCoder() option to the
> > transform
> > >> and
> > >> > > > >> >> propagating that coder to its sub-transforms. If the
> option
> > is
> > >> > > unset,
> > >> > > > >> >> that's fine for now.
> > >> > > > >> >> - As a result of the above, get rid of all setCoder()
> calls
> > in
> > >> > the
> > >> > > > Beam
> > >> > > > >> >> repo. The call will still be there, but it will just not
> be
> > >> used
> > >> > > > >> anywhere
> > >> > > > >> >> in the SDK or examples, and we can mark it deprecated.
> > >> > > > >> >> - Add guidance to PTransform Style Guide in line with the
> > >> above
> > >> > > > >> >>
> > >> > > > >> >> Does this sound like a good idea? I'm not sure how urgent
> it
> > >> > would
> > >> > > > be to
> > >> > > > >> >> actually do this, but I'd like to know whether people
> agree
> > >> that
> > >> > > this
> > >> > > > >> is a
> > >> > > > >> >> good goal in general.
> > >> > > > >> >>
> > >> > > > >> >>
> > >> > > > >> > --
> > >> > > > >> > Jean-Baptiste Onofré
> > >> > > > >> > jbono...@apache.org
> > >> > > > >> > http://blog.nanthrax.net
> > >> > > > >> > Talend - http://www.talend.com
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Reply via email to